From e0bdece5511c5b1db1c009697819bc42c342f0c3 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 13:33:25 -0500 Subject: [PATCH 01/73] random gitignores --- .gitignore | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/.gitignore b/.gitignore index c800ff1..76b1a43 100644 --- a/.gitignore +++ b/.gitignore @@ -24,3 +24,7 @@ pom.xml.versionsBackup .calva/ .lsp/ .vscode/ +.project +.classpath +ccw.repl.cmdhistory.prefs +nbactions.xml From 03b97e0b3e0ec329629bcbf76106658dce4a5d61 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 14:20:08 -0500 Subject: [PATCH 02/73] added core.async.flow --- src/main/clojure/clojure/core/async/flow.clj | 270 +++++++++++++++++ .../clojure/clojure/core/async/flow/impl.clj | 273 ++++++++++++++++++ .../clojure/core/async/flow/impl/graph.clj | 28 ++ .../clojure/clojure/core/async/flow/spi.clj | 95 ++++++ 4 files changed, 666 insertions(+) create mode 100644 src/main/clojure/clojure/core/async/flow.clj create mode 100644 src/main/clojure/clojure/core/async/flow/impl.clj create mode 100644 src/main/clojure/clojure/core/async/flow/impl/graph.clj create mode 100644 src/main/clojure/clojure/core/async/flow/spi.clj diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj new file mode 100644 index 0000000..e0c06dd --- /dev/null +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -0,0 +1,270 @@ +;; Copyright (c) Rich Hickey and contributors. All rights reserved. +;; The use and distribution terms for this software are covered by the +;; Eclipse Public License 1.0 (http://opensource.org/licenses/eclipse-1.0.php) +;; which can be found in the file epl-v10.html at the root of this distribution. +;; By using this software in any fashion, you are agreeing to be bound by +;; the terms of this license. +;; You must not remove this notice, or any other, from this software. + +(ns ^{:author "Rich Hickey"} + clojure.core.async.flow + "A library for building concurrent, event driven data processing + flows out of communication-free functions, while centralizing + control, reporting, execution and error handling. Built on core.async. + + The top-level construct is the flow, comprising: + a set of processes (generally, threads) - concurrent activities + a set of channels flowing data into and out of the processes + a set of channels for centralized control, reporting, error-handling, + and execution of the processes + + A flow is constructed from flow definition data which defines a + directed graph of processes and the connections between + them. Processes describe their I/O requirements and the + flow (library) itself creates channels and passes them to the + processes that requested them. See 'create-flow' for the + details. The flow definition provides a centralized place for policy + decisions regarding configuration, threading, buffering etc. + + It is expected that applications will rarely define processes but + instead use the API functions here, 'process' and 'step-process', + that implement the process protocol in terms of calls to ordinary + functions that include no communication or core.async code. In this + way the library helps you achieve a strict separation of your + application logic from its execution, communication, lifecycle and + monitoring. + + Note that at several points the library calls upon the user to + supply ids for processes, inputs, outputs etc. These should be + keywords. When a namespaced keyword is required it is explicitly + stated. This documentation refers to various keywords utilized by + the library itself as ::flow/xyz, weere ::flow is an alias for + clojure.core.async.flow + + A process is represented in the flow definition by an implementation + of spi/ProcLauncher that starts it. See the spi docs for + details." + + (:require + [clojure.core.async.flow.impl :as impl] + [clojure.core.async.flow.impl.graph :as g])) + +(defn create-flow + "Creates a flow from the supplied definition: a map containing the + keys :procs and :conns, and optionally :mixed-exec/:io-exec/:compute-exec + + :procs - a map of pid->proc-def + where proc-def is a map with keys :proc, :args, :chan-opts + + :proc - a function that starts a process + :args - a map of param->val which will be passed to the process ctor + :chan-opts - a map of in-or-out-id->{:keys [buf-or-n xform]}, where buf-or-n + and xform have their meanings per core.async/chan + the default is {:buf-or-n 10} + + :conns - a collection of [[from-pid outid] [to-pid inid]] tuples. + + Inputs and outputs support mutliple connections. When an output is + connected multiple times every connection will get every message, + as per a core.async/mult. + + :mixed-exec/:io-exec/:compute-exec -> ExecutorService + These can be used to specify the ExecutorService to use for the + corresonding context, in lieu of the lib defaults + + N.B. The flow is not started. See 'start'" + [def] (impl/create-flow def)) + +(defn start + "starts the entire flow from init values. The processes start paused. + Call resume or resume-proc to start flow. + returns {::flow/report-chan - a core.async chan for reading + ::flow/error-chan - a core.async chan for reading}" + [g] (g/start g)) + +(defn stop + "shuts down the flow, stopping all procsesses and closing the error + and report channels. The flow can be started again" + [g] (g/stop g)) + +(defn pause + "pauses a running flow" + [g] (g/pause g)) + +(defn resume + "resumes a paused flow" + [g] (g/resume g)) + +(defn ping + "pings all processes, which will put their status and state on the + report channel" + [g] (g/ping g)) + +(defn pause-proc + "pauses a process" + [g pid] (g/pause-proc g pid)) + +(defn resume-proc + "resumes a process" + [g pid] (g/resume-proc g pid)) + +(defn ping-proc + "pings the process, which will put its status and state on the report + channel" + [g pid] (g/ping-proc g pid)) + +(defn command-proc + "synchronously sends a process-specific command with the given id and + additional kvs to the process. The cmd-id must be ns-qualified with + a ns you own." + [g pid cmd-id more-kvs] (g/command-proc g pid cmd-id more-kvs)) + +(defn inject + "synchronously puts the messages on the channel corresponding to the + input or output of the process" + [g [pid io-id :as coord] msgs] (g/inject g coord msgs)) + +(defn process + "Given a map of functions (described below), returns a launcher that + creates a process compliant with the process protocol (see the + spi/ProcLauncher doc). The possible entries for process-impl-map + are :describe, :init, :transition, :transform and :inject. This is + the core facility for defining the logic for processes via ordinary + functions. + + :describe - required, () -> desc + where desc is a map with keys :params :ins and :outs, each of which + in turn is a map of keyword to doc string + + :params describes the initial arguments to setup the state for the function. + :ins enumerates the input[s], for which the flow will create channels + :outs enumerates the output[s], for which the flow may create channels. + + No key may be present in both :ins and :outs The ins/outs/params of f + will be the ins/outs/params of the process. describe may be called + by users to understand how to use the proc. It will also be called + by the impl in order to discover what channels are needed. + + :init - optional, (arg-map) -> initial-state + + init will be called once by the process to establish any + initial state. The arg-map will be a map of param->val, as supplied + in the flow def. init must be provided if 'describe' returns :params. + + :transition - optional, (state transition) -> state' + + transition will be called when the process makes a state transition, + transition being one of ::flow/resume, ::flow/pause or ::flow/stop + + With this fn a process impl can track changes and coordinate + resources, especially cleaning up any resources on :stop, since the + process will no longer be used following that. See the SPI for + details. state' will be the state supplied to subsequent calls. + + Exactly one of either :transform or :inject are required. + + :transform - (state in-name msg) -> [state' output] + where output is a map of outid->[msgs*] + + The transform fn will be called every time a message arrives at any + of the inputs. Output can be sent to none, any or all of the :outs + enumerated, and/or an input named by a [pid inid] tuple (e.g. for + reply-to), and/or to the ::flow/report output. A step need not + output at all (output or msgs can be empyt/nil), however an output _message_ + may never be nil (per core.async channels). state' will be the state + supplied to subsequent calls. + + :inject - (state) -> [state' output] + where output is a map of outid->[msgs*], per :transform + + The inject fn is used for sources - proc-impls that inject new data + into the flow by doing I/O with something external to the flow and + feeding that data to its outputs. A proc-impl specifying :inject may not + specify any :ins in its descriptor, as none but the ::flow/control channel + will be read. Instead, inject will be called every time through the + process loop, and will presumably do blocking or paced I/O to get + new data to return via its outputs. If it does blocking I/O it + should do so with a timeout so it can regularly return to the + process loop which can then look for control messages - it's fine + for inject to return with no output. Do not spin poll in the inject + fn. + + proc accepts an option map with keys: + :exec - one of :mixed, :io or :compute, default :mixed + :compute-timeout-ms - if :exec is :compute, this timeout (default 5000 msec) + will be used when getting the return from the future - see below + + The :compute context is not allowed for proc impls that + provide :inject (as I/O is presumed). + + In the :exec context of :mixed or :io, this dictates the type of + thread in which the process loop will run, _including its calls to + transform/inject_. + + When :io is specified transform/inject should not do extensive computation. + + When :compute is specified (only allowed for :transform), each call + to transform will be run in a separate thread. The process loop will + run in an :io context (since it no longer directly calls transform, + all it does is I/O) and it will submit transform to the :compute + executor then await (blocking, for compute-timeout-ms) the + completion of the future returned by the executor. If the future + times out it will be reported on ::flow/error. + + When :compute is specified transform must not block!" + ([process-impl-map] (process process-impl-map nil)) + ([process-impl-map {:keys [exec timeout-ms] + :or {exec :mixed, timeout-ms 5000} :as opts}] + (impl/proc process-impl-map opts))) + +(defn step-process + "Given a (e.g. communication-free) step function f of three + arities (described below), and the same opts as 'process', returns a + launcher that creates a process compliant with the process + protocol (see 'process'). + + The arities of f are: + + ()->desc + a function matching the semantics of process' :describe + + (arg-map)->initial-state + a function matching the semantics of process' :init + + (state in-name msg)->[state' output] + a function matching the semantics of process' :transform" + ([f] (step-process f nil)) + ([f opts] + (process {:describe f, :init f, :transform f} opts))) + +(defn futurize + "Takes a fn f and returns a fn that takes the same arguments as f and + immediately returns a future, having starting a thread of the + indicated type, or via the supplied executor, that invokes f with + those args and completes that future with its return. + + futurize accepts kwarg options: + :exec - one of :mixed, :io, :compute + or a j.u.c.ExecutorService object, + default :mixed" + [f & {:keys [exec] + :or {exec :mixed} :as opts}] + (impl/futurize f opts)) + +(defn lift*->step + "given a fn f taking one arg and returning a collection of non-nil + values, create a 'step' fn as needed by step-process, with one input + and one output (named :in and :out), and no state." + [f] + (fn + ([] {:params {} + :ins {:in (str "the argument to " f)} + :outs {:out (str "the return of " f)}}) + ([_] nil) + ([_ _ msg] [nil {:out (f msg)}]))) + +(defn lift1->step + "like lift*->step except taking a fn returning one value, which, when + nil, will yield no output." + [f] + (lift*->step #(when-some [m (f %)] (vector m)))) diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj new file mode 100644 index 0000000..e6362db --- /dev/null +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -0,0 +1,273 @@ +;; Copyright (c) Rich Hickey and contributors. All rights reserved. +;; The use and distribution terms for this software are covered by the +;; Eclipse Public License 1.0 (http://opensource.org/licenses/eclipse-1.0.php) +;; which can be found in the file epl-v10.html at the root of this distribution. +;; By using this software in any fashion, you are agreeing to be bound by +;; the terms of this license. +;; You must not remove this notice, or any other, from this software. + +(ns clojure.core.async.flow.impl + (:require [clojure.core.async :as async] + [clojure.core.async.flow :as-alias flow] + [clojure.core.async.flow.spi :as spi] + [clojure.core.async.flow.impl.graph :as graph]) + (:import [java.util.concurrent Future Executors ExecutorService TimeUnit] + [java.util.concurrent.locks ReentrantLock])) + +(set! *warn-on-reflection* true) + +;;TODO - something specific, e.g. make aware of JDK version and vthreads +(defonce mixed-exec clojure.lang.Agent/soloExecutor) +(defonce io-exec clojure.lang.Agent/soloExecutor) +(defonce compute-exec clojure.lang.Agent/pooledExecutor) + +(defn futurize ^Future [f {:keys [exec]}] + (fn [& args] + (^[Callable] ExecutorService/.submit + (case exec + :compute compute-exec + :io io-exec + :mixed mixed-exec + exec) + #(apply f args)))) + +(defn prep-proc [ret pid {:keys [proc, args, chan-opts] :or {chan-opts {}}}] + (let [{:keys [ins outs]} (spi/describe proc) + copts (fn [cs] + (zipmap (keys cs) (map #(chan-opts %) (keys cs)))) + inopts (copts ins) + outopts (copts outs)] + (when (or (some (partial contains? inopts) (keys outopts)) + (some (partial contains? outopts) (keys inopts))) + (throw (ex-info ":ins and :outs cannot share ids within a process" + {:pid pid :ins (keys inopts) :outs (keys outopts)}))) + (assoc ret pid {:pid pid :proc proc :ins inopts :outs outopts :args args}))) + +(defn create-flow + "see lib ns for docs" + [{:keys [procs conns mixed-exec io-exec compute-exec] + :or {mixed-exec mixed-exec, io-exec io-exec, compute-exec compute-exec} + :as desc}] + (let [lock (ReentrantLock.) + chans (atom nil) + execs {:mixed mixed-exec :io io-exec :compute compute-exec} + _ (assert (every? #(instance? ExecutorService %) (vals execs)) + "mixed-exe, io-exec and compute-exec must be ExecutorServices") + pdescs (reduce-kv prep-proc {} procs) + allopts (fn [iok] (into {} (mapcat #(map (fn [[k opts]] [[(:pid %) k] opts]) (iok %)) (vals pdescs)))) + inopts (allopts :ins) + outopts (allopts :outs) + set-conj (fnil conj #{}) + ;;out-coord->#{in-coords} + conn-map (reduce (fn [ret [out in :as conn]] + (if (and (contains? outopts out) + (contains? inopts in)) + (update ret out set-conj in) + (throw (ex-info "invalid connection" {:conn conn})))) + {} conns) + running-chans #(or (deref chans) (throw (Exception. "flow not running"))) + send-command (fn [command to] + (let [{:keys [control]} (running-chans)] + (async/>!! control #::flow{:command command :to to})))] + (reify + clojure.core.async.flow.impl.graph.Graph + (start [_] + (.lock lock) + (try + (if-let [{:keys [report error]} @chans] + {:report-chan report :error-chan error :already-running true} + (let [control-chan (async/chan 10) + control-mult (async/mult control-chan) + report-chan (async/chan (async/sliding-buffer 100)) + error-chan (async/chan (async/sliding-buffer 100)) + make-chan (fn [[[pid cid]{:keys [buf-or-n xform]}]] + (if xform + (async/chan + buf-or-n xform + (fn [ex] + (async/put! error-chan + #::flow{:ex ex, :pid pid, :cid cid, :xform xform}) + nil)) + (async/chan (or buf-or-n 10)))) + in-chans (zipmap (keys inopts) (map make-chan inopts)) + out-chans (zipmap (keys outopts) + (map (fn [[coord opts :as co]] + (let [conns (conn-map coord)] + (cond + (empty? conns) nil + ;;direct connect 1:1 + (= 1 (count conns)) (in-chans (first conns)) + :else (make-chan co)))) + outopts)) + ;;mults + _ (doseq [[out ins] conn-map] + (when (< 1 (count ins)) + (let [m (async/mult (out-chans out))] + (doseq [in ins] + (async/tap m (in-chans in)))))) + write-chan #(if-let [[_ c] (or (find in-chans %) (find out-chans %))] + c + (throw (ex-info "can't resolve channel with coord" {:coord %}))) + resolver (reify spi/Resolver + (get-write-chan [_ coord] + (write-chan coord)) + (get-exec [_ context] (execs context))) + start-proc + (fn [{:keys [pid proc args ins outs]}] + (try + (let [chan-map (fn [ks coll] (zipmap (keys ks) (map #(coll [pid %]) (keys ks)))) + control-tap (async/chan 10)] + (async/tap control-mult control-tap) + (spi/start proc {:pid pid :args args :resolver resolver + :ins (assoc (chan-map ins in-chans) + ::flow/control control-tap) + :outs (assoc (chan-map outs out-chans) + ::flow/error error-chan + ::flow/report report-chan)})) + (catch Throwable ex + (async/>!! control-chan #::flow{:command ::flow/stop :to ::flow/all}) + (throw ex))))] + (doseq [p (vals pdescs)] + (start-proc p)) + ;;the only connection to a running flow is via channels + (reset! chans {:control control-chan :resolver resolver + :report report-chan, :error error-chan + :ins in-chans, :outs out-chans}) + {:report-chan report-chan :error-chan error-chan})) + (finally (.unlock lock)))) + (stop [_] + (.lock lock) + (try + (when-let [{:keys [report error]} @chans] + (send-command ::flow/stop ::flow/all) + (async/close! error) + (async/close! report) + (reset! chans nil) + true) + (finally (.unlock lock)))) + (pause [_] (send-command ::flow/pause ::flow/all)) + (resume [_] (send-command ::flow/resume ::flow/all)) + (ping [_] (send-command ::flow/ping ::flow/all)) + + (pause-proc [_ pid] (send-command ::flow/pause pid)) + (resume-proc [_ pid] (send-command ::flow/resume pid)) + (ping-proc [_ pid] (send-command ::flow/ping pid)) + (command-proc [_ pid command kvs] + (assert (and (namespace command) (not= (namespace ::flow/command) (namespace command))) + "extension commands must be in your own namespace") + (let [{:keys [control]} (running-chans)] + (async/>!! control (merge kvs #::flow{:command command :to pid})))) + + (inject [_ coord msgs] + (let [{:keys [resolver]} (running-chans) + chan (spi/get-write-chan resolver coord)] + (doseq [m msgs] + (async/>!! chan m))))))) + +(defn handle-command + [pid pong status cmd] + (let [transition #::flow{:stop :exit, :resume :running, :pause :paused} + {::flow/keys [to command]} cmd] + (if (#{::flow/all pid} to) + (do + (when (= command ::flow/ping) (pong)) + (or (transition command) status)) + status))) + +(defn handle-transition + "when transition, returns maybe new state" + [transition status nstatus state] + (if (and transition (not= status nstatus)) + (transition state nstatus) + state)) + +(defn send-outputs [status state outputs outs resolver control handle-command transition] + (loop [nstatus status, nstate state, outputs (seq outputs)] + (if (or (nil? outputs) (= nstatus :exit)) + [nstatus nstate] + (let [[output msgs] (first outputs)] + (if-let [outc (or (outs output) (spi/get-write-chan resolver output))] + (let [[nstatus nstate] + (loop [nstatus nstatus, nstate nstate, msgs (seq msgs)] + (if (or (nil? msgs) (= nstatus :exit)) + [nstatus nstate] + (let [[v c] (async/alts!! + [control [outc (first msgs)]] + :priority true)] + (if (= c control) + (let [nnstatus (handle-command nstatus v) + nnstate (handle-transition transition nstatus nnstatus nstate)] + (recur nnstatus nnstate msgs)) + (recur nstatus nstate (next msgs))))))] + (recur nstatus nstate (next outputs))) + (recur nstatus nstate (next outputs))))))) + +(defn proc + "see lib ns for docs" + [{:keys [describe init transition transform inject] :as impl} {:keys [exec compute-timeout-ms]}] + ;;validate the preconditions + (assert (= 1 (count (keep identity [transform inject]))) "must provide exactly one of :transform or :inject") + (assert (not (and inject (= exec :compute))) "can't specify :inject and :compute") + (reify + clojure.core.protocols/Datafiable + (datafy [_] + (let [{:keys [params ins outs]} (describe)] + {:impl impl :params (-> params keys vec) :ins (-> ins keys vec) :outs (-> outs keys vec)})) + spi/ProcLauncher + (describe [_] + (let [{:keys [params ins] :as desc} (describe)] + (assert (not (and ins inject)) "can't specify :ins when :inject") + (assert (or (not params) init) "must have :init if :params") + desc)) + (start [_ {:keys [pid args ins outs resolver]}] + (let [comp? (= exec :compute) + transform (cond-> transform (= exec :compute) + #(.get (futurize transform {:exec (spi/get-exec resolver :compute)}) + compute-timeout-ms TimeUnit/MILLISECONDS)) + exs (spi/get-exec resolver (if (= exec :mixed) :mixed :io)) + io-id (zipmap (concat (vals ins) (vals outs)) (concat (keys ins) (keys outs))) + control (::flow/control ins) + ;;TODO rotate/randomize after control per normal alts? + read-chans (into [control] (-> ins (dissoc ::flow/control) vals)) + run + #(loop [status :paused, state (when init (init args)), count 0] + (let [pong (fn [] (async/>!! (outs ::flow/report) + #::flow{:report :ping, :pid pid, :status status + :state state, :count count})) + handle-command (partial handle-command pid pong) + [nstatus nstate count] + (try + (if (= status :paused) + (let [nstatus (handle-command status (async/!! (outs ::flow/error) + #::flow{:pid pid, :status status, :state state, + :count (inc count), :cid cid, :msg msg :op :step, :ex ex}) + [status state count]))))) + (catch Throwable ex + (async/>!! (outs ::flow/error) + #::flow{:pid pid, :status status, :state state, :count (inc count), :ex ex}) + [status state count]))] + (when-not (= nstatus :exit) ;;fall out + (recur nstatus nstate (long count)))))] + ((futurize run {:exec exs})))))) diff --git a/src/main/clojure/clojure/core/async/flow/impl/graph.clj b/src/main/clojure/clojure/core/async/flow/impl/graph.clj new file mode 100644 index 0000000..181a392 --- /dev/null +++ b/src/main/clojure/clojure/core/async/flow/impl/graph.clj @@ -0,0 +1,28 @@ +;; Copyright (c) Rich Hickey and contributors. All rights reserved. +;; The use and distribution terms for this software are covered by the +;; Eclipse Public License 1.0 (http://opensource.org/licenses/eclipse-1.0.php) +;; which can be found in the file epl-v10.html at the root of this distribution. +;; By using this software in any fashion, you are agreeing to be bound by +;; the terms of this license. +;; You must not remove this notice, or any other, from this software. + +(ns clojure.core.async.flow.impl.graph) + +(defprotocol Graph + (start [g] "starts the entire graph from init values. The processes start paused. + Call resume-all or resume-proc to start flow. + returns {:report-chan - a core.async chan for reading + :error-chan - a core.async chan for reading}") + (stop [g] "shuts down the graph, stopping all procs, can be started again") + (pause [g] "pauses a running graph") + (resume [g] "resumes a paused graph") + (ping [g] "pings all processes, which will put their status and state on the report channel") + + (pause-proc [g pid] "pauses a process") + (resume-proc [g pid] "resumes a process") + (ping-proc [g pid] "pings the process, which will put its status and state on the report channel") + (command-proc [g pid cmd-id more-kvs] "synchronously sends a process-specific command with the given id + and additional kvs to the process") + + (inject [g [pid io-id] msgs] "synchronously puts the messages on the channel + corresponding to the input or output of the process")) diff --git a/src/main/clojure/clojure/core/async/flow/spi.clj b/src/main/clojure/clojure/core/async/flow/spi.clj new file mode 100644 index 0000000..3bb4df4 --- /dev/null +++ b/src/main/clojure/clojure/core/async/flow/spi.clj @@ -0,0 +1,95 @@ +;; Copyright (c) Rich Hickey and contributors. All rights reserved. +;; The use and distribution terms for this software are covered by the +;; Eclipse Public License 1.0 (http://opensource.org/licenses/eclipse-1.0.php) +;; which can be found in the file epl-v10.html at the root of this distribution. +;; By using this software in any fashion, you are agreeing to be bound by +;; the terms of this license. +;; You must not remove this notice, or any other, from this software. + +(ns clojure.core.async.flow.spi) + +(defprotocol ProcLauncher + "Note - definine a ProcLauncher is an advanced feature and should not + be needed for ordinary use of the library. This protocol is for + creating new types of Processes that are not possible to create + with ::flow/process. + + A ProcLauncher is a constructor for a process, a thread of activity. + It has two functions - to describe the parameters and input/output + requirements of the process and to start it. The launcher should + acquire no resources, nor retain any connection to the started + process. A launcher may be called upon to start a process more than + once, and should start a new process each time start is called. + + The process launched process must obey the following: + + It must have 2 logical statuses, :paused and :running. In + the :paused status operation is suspended and no output is + produced. + + When the process starts it must be :paused + + Whenever it is reading or writing to any channel a process must use + alts!! and include a read of the ::flow/control channel, giving it + priority. + + Command messages sent over the ::flow/control channel have the keys: + ::flow/to - either ::flow/all or a process id + ::flow/command - ::flow/stop|pause|resume|ping or process-specific + + It must act upon any, and only, control messages whose ::flow/to key is its pid or ::flow/all + It must act upon the following values of ::flow/command: + + ::flow/stop - all resources should be cleaned up and any thread(s) + should exit ordinarily - there will be no more subsequent use + of the process. + ::flow/pause - enter the :paused status + ::flow/resume - enter the :running status and resume processing + ::flow/ping - emit a ping message (format TBD) to + the ::flow/report channel containing at least its pid and status + + A process can define and respond to other commands in its own namespace. + + A process should not transmit channel objects (use [pid io-id] data + coordinates instead) A process should not close channels + + Finally, if a process encounters an error it must report it on the + ::flow/error channel (format TBD) and attempt to continue, though it + may subsequently get a ::flow/stop command it must respect" + + (describe [p] + "returns a map with keys - :params, :ins and :outs, + each of which in turn is a map of keyword to docstring + + :params describes the initial arguments to setup the state for the process + :ins enumerates the input[s], for which the graph will create channels + :outs enumerates the output[s], for which the graph may create channels. + + describe may be called by users to understand how to use the + proc. It will also be called by the impl in order to discover what + channels are needed.") + + (start [p {:keys [pid args ins outs resolver]}] + "return ignored, called for the + effect of starting the process (typically, starting its thread) + + where: + + :pid - the id of the process in the graph, so that e.g. it can refer to itself in control, reporting etc + :args - a map of param->val, as supplied in the graph def + :ins - a map of in-id->readable-channel, plus the ::flow/control channel + :outs - a map of out-id->writeable-channel, plus the ::flow/error and ::flow/report channels + N.B. outputs may be nil if not connected + :resolver - an impl of spi/Resolver, which can be used to find + channels given their logical [pid cid] coordinates, as well as to + obtain ExecutorServices corresponding to the + logical :mixed/:io/:compute contexts")) + +(defprotocol Resolver + (get-write-chan [_ coord] + "Given a tuple of [pid cid], returns a core.async chan to + write to or nil (in which case the output should be dropped, + e.g. nothing is connected).") + (get-exec [_ context] + "returns the ExecutorService for the given context, one + of :mixed, :io, :compute")) From 5b086bcf163903514a641dcbf2600bdb5547dcfc Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 15:24:44 -0500 Subject: [PATCH 03/73] added channel status to ping replies --- src/main/clojure/clojure/core/async/flow.clj | 2 ++ .../clojure/clojure/core/async/flow/impl.clj | 24 +++++++++++++++---- 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj index e0c06dd..09553ac 100644 --- a/src/main/clojure/clojure/core/async/flow.clj +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -49,6 +49,8 @@ [clojure.core.async.flow.impl :as impl] [clojure.core.async.flow.impl.graph :as g])) +(set! *warn-on-reflection* true) + (defn create-flow "Creates a flow from the supplied definition: a map containing the keys :procs and :conns, and optionally :mixed-exec/:io-exec/:compute-exec diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index e6362db..5a2cb90 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -21,6 +21,17 @@ (defonce io-exec clojure.lang.Agent/soloExecutor) (defonce compute-exec clojure.lang.Agent/pooledExecutor) +(defn chan->data + [^clojure.core.async.impl.channels.ManyToManyChannel c] + (let [b (.buf c)] + {:buffer-type (if b + (-> b class .getSimpleName symbol) + :none) + :buffer-count (count b) + :put-count (count (.puts c)) + :take-count (count (.takes c)) + :closed? (clojure.core.async.impl.protocols/closed? c)})) + (defn futurize ^Future [f {:keys [exec]}] (fn [& args] (^[Callable] ExecutorService/.submit @@ -231,9 +242,14 @@ read-chans (into [control] (-> ins (dissoc ::flow/control) vals)) run #(loop [status :paused, state (when init (init args)), count 0] - (let [pong (fn [] (async/>!! (outs ::flow/report) - #::flow{:report :ping, :pid pid, :status status - :state state, :count count})) + (let [pong (fn [] + (let [pins (dissoc ins ::flow/control) + pouts (dissoc outs ::flow/error ::flow/report)] + (async/>!! (outs ::flow/report) + #::flow{:report :ping, :pid pid, :status status + :state state, :count count + :ins (zipmap (keys pins) (map chan->data (vals pins))) + :outs (zipmap (keys pouts) (map chan->data (vals pouts)))}))) handle-command (partial handle-command pid pong) [nstatus nstate count] (try @@ -262,7 +278,7 @@ (catch Throwable ex (async/>!! (outs ::flow/error) #::flow{:pid pid, :status status, :state state, - :count (inc count), :cid cid, :msg msg :op :step, :ex ex}) + :count (inc count), :cid cid, :msg msg :op :step, :ex ex}) [status state count]))))) (catch Throwable ex (async/>!! (outs ::flow/error) From bf3c034acb0b0499320a4d71355431fb8439f4b3 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 18:27:51 -0500 Subject: [PATCH 04/73] more docs on 'start' return, typos --- src/main/clojure/clojure/core/async/flow.clj | 20 ++++++++++++++------ 1 file changed, 14 insertions(+), 6 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj index 09553ac..c1b8833 100644 --- a/src/main/clojure/clojure/core/async/flow.clj +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -38,7 +38,7 @@ supply ids for processes, inputs, outputs etc. These should be keywords. When a namespaced keyword is required it is explicitly stated. This documentation refers to various keywords utilized by - the library itself as ::flow/xyz, weere ::flow is an alias for + the library itself as ::flow/xyz, where ::flow is an alias for clojure.core.async.flow A process is represented in the flow definition by an implementation @@ -79,9 +79,17 @@ (defn start "starts the entire flow from init values. The processes start paused. - Call resume or resume-proc to start flow. - returns {::flow/report-chan - a core.async chan for reading - ::flow/error-chan - a core.async chan for reading}" + Call 'resume' or 'resume-proc' to start flow. returns a map with keys: + + ::flow/report-chan - a core.async chan for reading.'ping' reponses + will show up here, as will any explicit ::flow/report outputs + from :transform/:inject + + ::flow/error-chan - a core.async chan for reading. Any (and only) + exceptions thrown anywhere on any thread inside a flow will appear + in maps sent here. There will at least be a ::flow/ex entry with the + exception, and may be additional keys for pid, state, status etc + depending on the context of the error." [g] (g/start g)) (defn stop @@ -99,7 +107,7 @@ (defn ping "pings all processes, which will put their status and state on the - report channel" + report channel returned from start" [g] (g/ping g)) (defn pause-proc @@ -112,7 +120,7 @@ (defn ping-proc "pings the process, which will put its status and state on the report - channel" + channel returned from start" [g pid] (g/ping-proc g pid)) (defn command-proc From 8e18e17c8a6ad579bc40701b335c46f1fa29f608 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 20:53:51 -0500 Subject: [PATCH 05/73] first cut at flow rationale and overview --- doc/flow.md | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) create mode 100644 doc/flow.md diff --git a/doc/flow.md b/doc/flow.md new file mode 100644 index 0000000..b48427c --- /dev/null +++ b/doc/flow.md @@ -0,0 +1,29 @@ +# core.async.flow # +## Rationale ## + +The [rationale](https://clojure.org/news/2013/06/28/clojure-clore-async-channels) for **core.async** says "There comes a time in all good programs when components or subsystems must stop communicating directly with one another." And core.async provides fundamental tools (channels) for doing that. + +But using core.async well, e.g. keeping your I/O out of your computational logic, requires discipline and architectural savvy, and to do so consistently throughout an application or organization, conventions. Given channels, many architectural decisions remain regarding thread execution, backpressure, error handling etc. And often the topology of your network of communicating processes *emerges* out of the flow of control of your program as various pieces of code create threads and wire channels together, interleaved with computation, making it difficult to see the topology or administer it in one place. + +The fundamental objective of __core.async.flow__ is to enable a strict separation of your application logic from its topology, execution, communication, lifecycle, monitoring and error handling, all of which are provided by and centralized in, c.a.flow, yielding more consistent, robust, testable, observable and operable systems. + +## Overview ## + +__core.async.flow__ provides *concrete* implementations of two more abstractions - the '__process__' - a thread of activity, and the '__flow__' - a directed graph of processes communicating via channels. A single data structure describes your flow topology, and has all of the settings for threads, channels etc. A process accepts data from and provides data to channels. The process implementation in c.a.flow handles all channel I/O, thread lifecycle and coordination with the flow graph. + +All you need to do in you application is: + +1. Define ordinary, often pure, data->data functions that the processes will run in their inner loop to do the *computational* part of processing messages. These functions do not handle channels or threads or lifecycle, and do not even know they are running in a flow. They can be tested in isolation, and hot-reloaded. If they encounter a problem they can, and should, just throw an exception. The process will take care of it from there. + +2. Define a flow by creating a data structure that enumerates the processes and the connections between their inputs and outputs, as well as various configuration settings for both. + +With these application inputs, c.a.flow does the rest. It inquires of the processes what channels they require, creates those channels, then instantiates the processes making all of the channel connections between them. The processes in turn start threads (in fully user-configurable thread pools), await inputs, monitor the admin control channel, and when inputs arrive make a data->data calls to your application logic, taking the return from that and sending it to the designated output channels. The processes follow a protocol used by the flow to do lifecycle management and error handling. + +Once you've created a flow, the API provides functions to start/stop(shutdown) the flow, and to pause/resume both the flow and individual processes, to ping processes to get their state and that of their connected channels, to inject data into any point in the graph etc. The flow provides channels containing the ordinary monitoring/reporting stream and, separately, the error stream. + +The library provides many more details and features, including the ability to create, via ordinary functions, processes that act as __sources__ (of data from outside the flow) or __sinks__ (to recipients outside the flow) so you can situate your flow in a broader context while still coordinating resource management with the flow lifecycle. + +I hope __core.async.flow__ enables you to write more robust and smaller applications, with more focus on your domain logic and less on plumbing. + +Rich Hickey +1/2025 From 313569a932dffd6c683438909926a719d80a5d04 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Wed, 15 Jan 2025 20:59:58 -0500 Subject: [PATCH 06/73] typo --- doc/flow.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/doc/flow.md b/doc/flow.md index b48427c..c24c697 100644 --- a/doc/flow.md +++ b/doc/flow.md @@ -17,7 +17,7 @@ All you need to do in you application is: 2. Define a flow by creating a data structure that enumerates the processes and the connections between their inputs and outputs, as well as various configuration settings for both. -With these application inputs, c.a.flow does the rest. It inquires of the processes what channels they require, creates those channels, then instantiates the processes making all of the channel connections between them. The processes in turn start threads (in fully user-configurable thread pools), await inputs, monitor the admin control channel, and when inputs arrive make a data->data calls to your application logic, taking the return from that and sending it to the designated output channels. The processes follow a protocol used by the flow to do lifecycle management and error handling. +With these application inputs, c.a.flow does the rest. It inquires of the processes what channels they require, creates those channels, then instantiates the processes making all of the channel connections between them. The processes in turn start threads (in fully user-configurable thread pools), await inputs, monitor the admin control channel, and when inputs arrive make data->data calls to your application logic, taking the return from that and sending it to the designated output channels. The processes follow a protocol used by the flow to do lifecycle management and error handling. Once you've created a flow, the API provides functions to start/stop(shutdown) the flow, and to pause/resume both the flow and individual processes, to ping processes to get their state and that of their connected channels, to inject data into any point in the graph etc. The flow provides channels containing the ordinary monitoring/reporting stream and, separately, the error stream. From 3ee4a34f474334e8f6d6ed9dc348767878280936 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Thu, 16 Jan 2025 16:17:21 -0500 Subject: [PATCH 07/73] prevent nil messages from messing up alts --- .gitignore | 1 + src/main/clojure/clojure/core/async/flow/impl.clj | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 76b1a43..033484d 100644 --- a/.gitignore +++ b/.gitignore @@ -28,3 +28,4 @@ pom.xml.versionsBackup .classpath ccw.repl.cmdhistory.prefs nbactions.xml +.DS_Store diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index 5a2cb90..8332b68 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -202,8 +202,9 @@ (loop [nstatus nstatus, nstate nstate, msgs (seq msgs)] (if (or (nil? msgs) (= nstatus :exit)) [nstatus nstate] - (let [[v c] (async/alts!! - [control [outc (first msgs)]] + (let [m (if-some [m (first msgs)] m (throw "messages must be non-nil")) + [v c] (async/alts!! + [control [outc m]] :priority true)] (if (= c control) (let [nnstatus (handle-command nstatus v) From 17b05383302c490165cfe6be31b3a2605de0ba20 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Thu, 16 Jan 2025 16:31:38 -0500 Subject: [PATCH 08/73] :inject in process fn-map is now :introduce, more accurate and avoids confusion with flow/inject --- src/main/clojure/clojure/core/async/flow.clj | 22 +++++++++---------- .../clojure/clojure/core/async/flow/impl.clj | 14 ++++++------ 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj index c1b8833..3beb519 100644 --- a/src/main/clojure/clojure/core/async/flow.clj +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -83,7 +83,7 @@ ::flow/report-chan - a core.async chan for reading.'ping' reponses will show up here, as will any explicit ::flow/report outputs - from :transform/:inject + from :transform/:introduce ::flow/error-chan - a core.async chan for reading. Any (and only) exceptions thrown anywhere on any thread inside a flow will appear @@ -138,7 +138,7 @@ "Given a map of functions (described below), returns a launcher that creates a process compliant with the process protocol (see the spi/ProcLauncher doc). The possible entries for process-impl-map - are :describe, :init, :transition, :transform and :inject. This is + are :describe, :init, :transition, :transform and :introduce. This is the core facility for defining the logic for processes via ordinary functions. @@ -171,7 +171,7 @@ process will no longer be used following that. See the SPI for details. state' will be the state supplied to subsequent calls. - Exactly one of either :transform or :inject are required. + Exactly one of either :transform or :introduce are required. :transform - (state in-name msg) -> [state' output] where output is a map of outid->[msgs*] @@ -184,19 +184,19 @@ may never be nil (per core.async channels). state' will be the state supplied to subsequent calls. - :inject - (state) -> [state' output] + :introduce - (state) -> [state' output] where output is a map of outid->[msgs*], per :transform - The inject fn is used for sources - proc-impls that inject new data + The introduce fn is used for sources - proc-impls that introduce new data into the flow by doing I/O with something external to the flow and - feeding that data to its outputs. A proc-impl specifying :inject may not + feeding that data to its outputs. A proc-impl specifying :introduce may not specify any :ins in its descriptor, as none but the ::flow/control channel - will be read. Instead, inject will be called every time through the + will be read. Instead, introduce will be called every time through the process loop, and will presumably do blocking or paced I/O to get new data to return via its outputs. If it does blocking I/O it should do so with a timeout so it can regularly return to the process loop which can then look for control messages - it's fine - for inject to return with no output. Do not spin poll in the inject + for introduce to return with no output. Do not spin poll in the introduce fn. proc accepts an option map with keys: @@ -205,13 +205,13 @@ will be used when getting the return from the future - see below The :compute context is not allowed for proc impls that - provide :inject (as I/O is presumed). + provide :introduce (as I/O is presumed). In the :exec context of :mixed or :io, this dictates the type of thread in which the process loop will run, _including its calls to - transform/inject_. + transform/introduce_. - When :io is specified transform/inject should not do extensive computation. + When :io is specified transform/introduce should not do extensive computation. When :compute is specified (only allowed for :transform), each call to transform will be run in a separate thread. The process loop will diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index 8332b68..fdbf4db 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -202,7 +202,7 @@ (loop [nstatus nstatus, nstate nstate, msgs (seq msgs)] (if (or (nil? msgs) (= nstatus :exit)) [nstatus nstate] - (let [m (if-some [m (first msgs)] m (throw "messages must be non-nil")) + (let [m (if-some [m (first msgs)] m (throw (Exception. "messages must be non-nil"))) [v c] (async/alts!! [control [outc m]] :priority true)] @@ -216,10 +216,10 @@ (defn proc "see lib ns for docs" - [{:keys [describe init transition transform inject] :as impl} {:keys [exec compute-timeout-ms]}] + [{:keys [describe init transition transform introduce] :as impl} {:keys [exec compute-timeout-ms]}] ;;validate the preconditions - (assert (= 1 (count (keep identity [transform inject]))) "must provide exactly one of :transform or :inject") - (assert (not (and inject (= exec :compute))) "can't specify :inject and :compute") + (assert (= 1 (count (keep identity [transform introduce]))) "must provide exactly one of :transform or :introduce") + (assert (not (and introduce (= exec :compute))) "can't specify :introduce and :compute") (reify clojure.core.protocols/Datafiable (datafy [_] @@ -228,7 +228,7 @@ spi/ProcLauncher (describe [_] (let [{:keys [params ins] :as desc} (describe)] - (assert (not (and ins inject)) "can't specify :ins when :inject") + (assert (not (and ins introduce)) "can't specify :ins when :introduce") (assert (or (not params) init) "must have :init if :params") desc)) (start [_ {:keys [pid args ins outs resolver]}] @@ -261,7 +261,7 @@ ;;:running (let [[msg c] (if transform (async/alts!! read-chans :priority true) - ;;inject + ;;introduce (when-let [msg (async/poll! control)] [msg control])) cid (io-id c)] @@ -272,7 +272,7 @@ (try (let [[nstate outputs] (if transform (transform state cid msg) - (inject state)) + (introduce state)) [nstatus nstate] (send-outputs status nstate outputs outs resolver control handle-command transition)] [nstatus nstate (inc count)]) From 6d9fc5a6b4e9d4cef8b51725aa9b1b47029d72ba Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Fri, 17 Jan 2025 12:48:59 -0600 Subject: [PATCH 09/73] use pre-1.12 compatible interop syntax --- src/main/clojure/clojure/core/async/flow/impl.clj | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index fdbf4db..e5473a0 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -34,13 +34,12 @@ (defn futurize ^Future [f {:keys [exec]}] (fn [& args] - (^[Callable] ExecutorService/.submit - (case exec - :compute compute-exec - :io io-exec - :mixed mixed-exec - exec) - #(apply f args)))) + (let [^ExecutorService exec (case exec + :compute compute-exec + :io io-exec + :mixed mixed-exec + exec)] + (.submit exec ^Callable #(apply f args))))) (defn prep-proc [ret pid {:keys [proc, args, chan-opts] :or {chan-opts {}}}] (let [{:keys [ins outs]} (spi/describe proc) From 103393bc0552243dcbe7d2fe2fc20313255950c7 Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Fri, 17 Jan 2025 15:17:16 -0500 Subject: [PATCH 10/73] :exec option -> :workload, proc-fn :describe can return :workflow, process opt overridesflow definition -> config, no use of Clojure 1.12 features so 1.11 compatible --- src/main/clojure/clojure/core/async/flow.clj | 82 +++++---- .../clojure/clojure/core/async/flow/impl.clj | 160 +++++++++--------- .../clojure/clojure/core/async/flow/spi.clj | 2 +- 3 files changed, 128 insertions(+), 116 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj index 3beb519..4a957f2 100644 --- a/src/main/clojure/clojure/core/async/flow.clj +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -8,7 +8,10 @@ (ns ^{:author "Rich Hickey"} clojure.core.async.flow - "A library for building concurrent, event driven data processing + " + Note - Alpha, work-in-progress, names and other details are in flux + + A library for building concurrent, event driven data processing flows out of communication-free functions, while centralizing control, reporting, execution and error handling. Built on core.async. @@ -18,21 +21,21 @@ a set of channels for centralized control, reporting, error-handling, and execution of the processes - A flow is constructed from flow definition data which defines a + A flow is constructed from flow configuration data which defines a directed graph of processes and the connections between them. Processes describe their I/O requirements and the flow (library) itself creates channels and passes them to the processes that requested them. See 'create-flow' for the - details. The flow definition provides a centralized place for policy - decisions regarding configuration, threading, buffering etc. + details. The flow configuration provides a centralized place for + policy decisions regarding process settings, threading, buffering etc. - It is expected that applications will rarely define processes but - instead use the API functions here, 'process' and 'step-process', - that implement the process protocol in terms of calls to ordinary - functions that include no communication or core.async code. In this - way the library helps you achieve a strict separation of your - application logic from its execution, communication, lifecycle and - monitoring. + It is expected that applications will rarely define instances of the + process protocol but instead use the API functions here, 'process' + and 'step-process', that implement the process protocol in terms of + calls to ordinary functions that might include no communication or + core.async code. In this way the library helps you achieve a strict + separation of your application logic from its execution, + communication, lifecycle, error handling and monitoring. Note that at several points the library calls upon the user to supply ids for processes, inputs, outputs etc. These should be @@ -52,7 +55,7 @@ (set! *warn-on-reflection* true) (defn create-flow - "Creates a flow from the supplied definition: a map containing the + "Creates a flow from the supplied configuration: a map containing the keys :procs and :conns, and optionally :mixed-exec/:io-exec/:compute-exec :procs - a map of pid->proc-def @@ -66,26 +69,26 @@ :conns - a collection of [[from-pid outid] [to-pid inid]] tuples. - Inputs and outputs support mutliple connections. When an output is + Inputs and outputs support multiple connections. When an output is connected multiple times every connection will get every message, as per a core.async/mult. :mixed-exec/:io-exec/:compute-exec -> ExecutorService These can be used to specify the ExecutorService to use for the - corresonding context, in lieu of the lib defaults + corresonding workflow, in lieu of the lib defaults. N.B. The flow is not started. See 'start'" - [def] (impl/create-flow def)) + [config] (impl/create-flow config)) (defn start "starts the entire flow from init values. The processes start paused. Call 'resume' or 'resume-proc' to start flow. returns a map with keys: - ::flow/report-chan - a core.async chan for reading.'ping' reponses + :report-chan - a core.async chan for reading.'ping' reponses will show up here, as will any explicit ::flow/report outputs from :transform/:introduce - ::flow/error-chan - a core.async chan for reading. Any (and only) + :error-chan - a core.async chan for reading. Any (and only) exceptions thrown anywhere on any thread inside a flow will appear in maps sent here. There will at least be a ::flow/ex entry with the exception, and may be additional keys for pid, state, status etc @@ -144,16 +147,23 @@ :describe - required, () -> desc where desc is a map with keys :params :ins and :outs, each of which - in turn is a map of keyword to doc string + in turn is a map of keyword to doc string, and :workload with + possible values of :mixed :io :compute. All entries in the describe + return map are optional. :params describes the initial arguments to setup the state for the function. :ins enumerates the input[s], for which the flow will create channels :outs enumerates the output[s], for which the flow may create channels. + :workload - describes the nature of the workload, one of :mixed :io or :compute + an :io workload should not do extended computation + a :compute workload should never block - No key may be present in both :ins and :outs The ins/outs/params of f - will be the ins/outs/params of the process. describe may be called - by users to understand how to use the proc. It will also be called - by the impl in order to discover what channels are needed. + No key may be present in both :ins and :outs, allowing for a uniform + channel coordinate system of [:process-id :channel-id]. The + ins/outs/params returned will be the ins/outs/params of the + process. describe may be called by users to understand how to use + the proc. It will also be called by the impl in order to discover + what channels are needed. :init - optional, (arg-map) -> initial-state @@ -199,15 +209,19 @@ for introduce to return with no output. Do not spin poll in the introduce fn. - proc accepts an option map with keys: - :exec - one of :mixed, :io or :compute, default :mixed - :compute-timeout-ms - if :exec is :compute, this timeout (default 5000 msec) + process accepts an option map with keys: + :workflow - one of :mixed, :io or :compute + :compute-timeout-ms - if :workflow is :compute, this timeout (default 5000 msec) will be used when getting the return from the future - see below - The :compute context is not allowed for proc impls that + A :workflow supplied as an option to process will override + any :workflow returned by the :describe fn of the process. If neither + are provded the default is :mixed. + + The :compute workload is not allowed for proc impls that provide :introduce (as I/O is presumed). - In the :exec context of :mixed or :io, this dictates the type of + In the :workflow context of :mixed or :io, this dictates the type of thread in which the process loop will run, _including its calls to transform/introduce_. @@ -223,8 +237,8 @@ When :compute is specified transform must not block!" ([process-impl-map] (process process-impl-map nil)) - ([process-impl-map {:keys [exec timeout-ms] - :or {exec :mixed, timeout-ms 5000} :as opts}] + ([process-impl-map {:keys [workflow timeout-ms] + :or {timeout-ms 5000} :as opts}] (impl/proc process-impl-map opts))) (defn step-process @@ -248,13 +262,13 @@ (process {:describe f, :init f, :transform f} opts))) (defn futurize - "Takes a fn f and returns a fn that takes the same arguments as f and - immediately returns a future, having starting a thread of the - indicated type, or via the supplied executor, that invokes f with - those args and completes that future with its return. + "Takes a fn f and returns a fn that takes the same arguments as f + and immediately returns a future, having started a thread for the + indicated workload, or via the supplied executor, that invokes f + with those args and completes that future with its return. futurize accepts kwarg options: - :exec - one of :mixed, :io, :compute + :exec - one of the workloads :mixed, :io, :compute or a j.u.c.ExecutorService object, default :mixed" [f & {:keys [exec] diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index fdbf4db..89c51bf 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -34,13 +34,12 @@ (defn futurize ^Future [f {:keys [exec]}] (fn [& args] - (^[Callable] ExecutorService/.submit - (case exec - :compute compute-exec - :io io-exec - :mixed mixed-exec - exec) - #(apply f args)))) + (let [^ExecutorService e (case exec + :compute compute-exec + :io io-exec + :mixed mixed-exec + exec)] + (.submit e ^Callable #(apply f args))))) (defn prep-proc [ret pid {:keys [proc, args, chan-opts] :or {chan-opts {}}}] (let [{:keys [ins outs]} (spi/describe proc) @@ -57,8 +56,7 @@ (defn create-flow "see lib ns for docs" [{:keys [procs conns mixed-exec io-exec compute-exec] - :or {mixed-exec mixed-exec, io-exec io-exec, compute-exec compute-exec} - :as desc}] + :or {mixed-exec mixed-exec, io-exec io-exec, compute-exec compute-exec}}] (let [lock (ReentrantLock.) chans (atom nil) execs {:mixed mixed-exec :io io-exec :compute compute-exec} @@ -216,75 +214,75 @@ (defn proc "see lib ns for docs" - [{:keys [describe init transition transform introduce] :as impl} {:keys [exec compute-timeout-ms]}] - ;;validate the preconditions - (assert (= 1 (count (keep identity [transform introduce]))) "must provide exactly one of :transform or :introduce") - (assert (not (and introduce (= exec :compute))) "can't specify :introduce and :compute") - (reify - clojure.core.protocols/Datafiable - (datafy [_] - (let [{:keys [params ins outs]} (describe)] - {:impl impl :params (-> params keys vec) :ins (-> ins keys vec) :outs (-> outs keys vec)})) - spi/ProcLauncher - (describe [_] - (let [{:keys [params ins] :as desc} (describe)] - (assert (not (and ins introduce)) "can't specify :ins when :introduce") - (assert (or (not params) init) "must have :init if :params") - desc)) - (start [_ {:keys [pid args ins outs resolver]}] - (let [comp? (= exec :compute) - transform (cond-> transform (= exec :compute) - #(.get (futurize transform {:exec (spi/get-exec resolver :compute)}) - compute-timeout-ms TimeUnit/MILLISECONDS)) - exs (spi/get-exec resolver (if (= exec :mixed) :mixed :io)) - io-id (zipmap (concat (vals ins) (vals outs)) (concat (keys ins) (keys outs))) - control (::flow/control ins) - ;;TODO rotate/randomize after control per normal alts? - read-chans (into [control] (-> ins (dissoc ::flow/control) vals)) - run - #(loop [status :paused, state (when init (init args)), count 0] - (let [pong (fn [] - (let [pins (dissoc ins ::flow/control) - pouts (dissoc outs ::flow/error ::flow/report)] - (async/>!! (outs ::flow/report) - #::flow{:report :ping, :pid pid, :status status - :state state, :count count - :ins (zipmap (keys pins) (map chan->data (vals pins))) - :outs (zipmap (keys pouts) (map chan->data (vals pouts)))}))) - handle-command (partial handle-command pid pong) - [nstatus nstate count] - (try - (if (= status :paused) - (let [nstatus (handle-command status (async/!! (outs ::flow/error) - #::flow{:pid pid, :status status, :state state, - :count (inc count), :cid cid, :msg msg :op :step, :ex ex}) - [status state count]))))) - (catch Throwable ex - (async/>!! (outs ::flow/error) - #::flow{:pid pid, :status status, :state state, :count (inc count), :ex ex}) - [status state count]))] - (when-not (= nstatus :exit) ;;fall out - (recur nstatus nstate (long count)))))] - ((futurize run {:exec exs})))))) + [{:keys [describe init transition transform introduce] :as impl} {:keys [workflow compute-timeout-ms]}] + (assert (= 1 (count (keep identity [transform introduce]))) "must provide exactly one of :transform or :introduce") + (let [{:keys [params ins] :as desc} (describe) + workflow (or workflow (:workflow desc) :mixed)] + (assert (not (and ins introduce)) "can't specify :ins when :introduce") + (assert (or (not params) init) "must have :init if :params") + (assert (not (and introduce (= workflow :compute))) "can't specify :introduce and :compute") + (reify + clojure.core.protocols/Datafiable + (datafy [_] + (let [{:keys [params ins outs]} desc] + {:impl impl :params (-> params keys vec) :ins (-> ins keys vec) :outs (-> outs keys vec)})) + spi/ProcLauncher + (describe [_] desc) + (start [_ {:keys [pid args ins outs resolver]}] + (let [comp? (= workflow :compute) + transform (cond-> transform (= workflow :compute) + #(.get (futurize transform {:exec (spi/get-exec resolver :compute)}) + compute-timeout-ms TimeUnit/MILLISECONDS)) + exs (spi/get-exec resolver (if (= workflow :mixed) :mixed :io)) + io-id (zipmap (concat (vals ins) (vals outs)) (concat (keys ins) (keys outs))) + control (::flow/control ins) + ;;TODO rotate/randomize after control per normal alts? + read-chans (into [control] (-> ins (dissoc ::flow/control) vals)) + run + #(loop [status :paused, state (when init (init args)), count 0] + (let [pong (fn [] + (let [pins (dissoc ins ::flow/control) + pouts (dissoc outs ::flow/error ::flow/report)] + (async/>!! (outs ::flow/report) + #::flow{:report :ping, :pid pid, :status status + :state state, :count count + :ins (zipmap (keys pins) (map chan->data (vals pins))) + :outs (zipmap (keys pouts) (map chan->data (vals pouts)))}))) + handle-command (partial handle-command pid pong) + [nstatus nstate count] + (try + (if (= status :paused) + (let [nstatus (handle-command status (async/!! (outs ::flow/error) + #::flow{:pid pid, :status status, :state state, + :count (inc count), :cid cid, :msg msg :op :step, :ex ex}) + [status state count]))))) + (catch Throwable ex + (async/>!! (outs ::flow/error) + #::flow{:pid pid, :status status, :state state, :count (inc count), :ex ex}) + [status state count]))] + (when-not (= nstatus :exit) ;;fall out + (recur nstatus nstate (long count)))))] + ((futurize run {:exec exs}))))))) diff --git a/src/main/clojure/clojure/core/async/flow/spi.clj b/src/main/clojure/clojure/core/async/flow/spi.clj index 3bb4df4..e86a66a 100644 --- a/src/main/clojure/clojure/core/async/flow/spi.clj +++ b/src/main/clojure/clojure/core/async/flow/spi.clj @@ -9,7 +9,7 @@ (ns clojure.core.async.flow.spi) (defprotocol ProcLauncher - "Note - definine a ProcLauncher is an advanced feature and should not + "Note - defining a ProcLauncher is an advanced feature and should not be needed for ordinary use of the library. This protocol is for creating new types of Processes that are not possible to create with ::flow/process. From c5a524c6607a792bc51a81f727e7d24ef4cae23b Mon Sep 17 00:00:00 2001 From: Rich Hickey Date: Fri, 17 Jan 2025 15:59:03 -0500 Subject: [PATCH 11/73] s/workflow/workload --- src/main/clojure/clojure/core/async/flow.clj | 14 +++++++------- src/main/clojure/clojure/core/async/flow/impl.clj | 12 ++++++------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/main/clojure/clojure/core/async/flow.clj b/src/main/clojure/clojure/core/async/flow.clj index 4a957f2..38d8943 100644 --- a/src/main/clojure/clojure/core/async/flow.clj +++ b/src/main/clojure/clojure/core/async/flow.clj @@ -75,7 +75,7 @@ :mixed-exec/:io-exec/:compute-exec -> ExecutorService These can be used to specify the ExecutorService to use for the - corresonding workflow, in lieu of the lib defaults. + corresonding workload, in lieu of the lib defaults. N.B. The flow is not started. See 'start'" [config] (impl/create-flow config)) @@ -210,18 +210,18 @@ fn. process accepts an option map with keys: - :workflow - one of :mixed, :io or :compute - :compute-timeout-ms - if :workflow is :compute, this timeout (default 5000 msec) + :workload - one of :mixed, :io or :compute + :compute-timeout-ms - if :workload is :compute, this timeout (default 5000 msec) will be used when getting the return from the future - see below - A :workflow supplied as an option to process will override - any :workflow returned by the :describe fn of the process. If neither + A :workload supplied as an option to process will override + any :workload returned by the :describe fn of the process. If neither are provded the default is :mixed. The :compute workload is not allowed for proc impls that provide :introduce (as I/O is presumed). - In the :workflow context of :mixed or :io, this dictates the type of + In the :workload context of :mixed or :io, this dictates the type of thread in which the process loop will run, _including its calls to transform/introduce_. @@ -237,7 +237,7 @@ When :compute is specified transform must not block!" ([process-impl-map] (process process-impl-map nil)) - ([process-impl-map {:keys [workflow timeout-ms] + ([process-impl-map {:keys [workload timeout-ms] :or {timeout-ms 5000} :as opts}] (impl/proc process-impl-map opts))) diff --git a/src/main/clojure/clojure/core/async/flow/impl.clj b/src/main/clojure/clojure/core/async/flow/impl.clj index 89c51bf..cc264d0 100644 --- a/src/main/clojure/clojure/core/async/flow/impl.clj +++ b/src/main/clojure/clojure/core/async/flow/impl.clj @@ -214,13 +214,13 @@ (defn proc "see lib ns for docs" - [{:keys [describe init transition transform introduce] :as impl} {:keys [workflow compute-timeout-ms]}] + [{:keys [describe init transition transform introduce] :as impl} {:keys [workload compute-timeout-ms]}] (assert (= 1 (count (keep identity [transform introduce]))) "must provide exactly one of :transform or :introduce") (let [{:keys [params ins] :as desc} (describe) - workflow (or workflow (:workflow desc) :mixed)] + workload (or workload (:workload desc) :mixed)] (assert (not (and ins introduce)) "can't specify :ins when :introduce") (assert (or (not params) init) "must have :init if :params") - (assert (not (and introduce (= workflow :compute))) "can't specify :introduce and :compute") + (assert (not (and introduce (= workload :compute))) "can't specify :introduce and :compute") (reify clojure.core.protocols/Datafiable (datafy [_] @@ -229,11 +229,11 @@ spi/ProcLauncher (describe [_] desc) (start [_ {:keys [pid args ins outs resolver]}] - (let [comp? (= workflow :compute) - transform (cond-> transform (= workflow :compute) + (let [comp? (= workload :compute) + transform (cond-> transform (= workload :compute) #(.get (futurize transform {:exec (spi/get-exec resolver :compute)}) compute-timeout-ms TimeUnit/MILLISECONDS)) - exs (spi/get-exec resolver (if (= workflow :mixed) :mixed :io)) + exs (spi/get-exec resolver (if (= workload :mixed) :mixed :io)) io-id (zipmap (concat (vals ins) (vals outs)) (concat (keys ins) (keys outs))) control (::flow/control ins) ;;TODO rotate/randomize after control per normal alts? From 55225fa05a7cbfc184bd711447f8d92d101e5f33 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Tue, 21 Jan 2025 12:29:05 -0600 Subject: [PATCH 12/73] add codox task --- deps.edn | 8 ++++++++ src/main/clojure/clojure/core/async.clj | 2 ++ src/test/clojure/clojure/core/async_test.clj | 7 +++++++ 3 files changed, 17 insertions(+) diff --git a/deps.edn b/deps.edn index 91e5dfb..ad96f46 100644 --- a/deps.edn +++ b/deps.edn @@ -22,4 +22,12 @@ ;; clj -M:outdated :outdated {:extra-deps {com.github.liquidz/antq {:mvn/version "2.11.1250"}} :main-opts ["-m" "antq.core"]} + + ;; Regenerate docs in docs/ + ;; clj -X:docs + :docs {:extra-deps {codox/codox {:mvn/version "0.10.8"}} + :exec-fn codox.main/generate-docs + :exec-args {:source-paths ["src/main/clojure"] + :namespaces [clojure.core.async clojure.core.async.flow clojure.core.async.flow.spi] + :output-path "apidocs"}} }} diff --git a/src/main/clojure/clojure/core/async.clj b/src/main/clojure/clojure/core/async.clj index 8b40571..de90e04 100644 --- a/src/main/clojure/clojure/core/async.clj +++ b/src/main/clojure/clojure/core/async.clj @@ -277,6 +277,8 @@ to catch and handle." "returns derefable [val port] if immediate, nil if enqueued" [fret ports opts] (assert (pos? (count ports)) "alts must have at least one channel operation") + (doseq [port ports :when (vector? port)] + (assert (not (nil? (nth port 1))) "alts can't put nil on channel")) (let [flag (alt-flag) ports (vec ports) ;; ensure vector for indexed nth n (count ports) diff --git a/src/test/clojure/clojure/core/async_test.clj b/src/test/clojure/clojure/core/async_test.clj index 017ba7d..866808a 100644 --- a/src/test/clojure/clojure/core/async_test.clj +++ b/src/test/clojure/clojure/core/async_test.clj @@ -465,3 +465,10 @@ :ok (catch AssertionError e :ko)))))) + +(deftest test-alts-put-nil-invalid + (is + (thrown? AssertionError + (let [c1 (a/chan) + c2 (a/chan)] + (a/alts!! [c1 [c2 nil]]))))) \ No newline at end of file From 1022d370e4c862d6a910aa1bf45c1789120e7e59 Mon Sep 17 00:00:00 2001 From: Alex Miller Date: Tue, 21 Jan 2025 12:29:29 -0600 Subject: [PATCH 13/73] add apidocs --- apidocs/clojure.core.async.flow.html | 206 +++++++++ apidocs/clojure.core.async.flow.spi.html | 74 +++ apidocs/clojure.core.async.html | 270 +++++++++++ apidocs/css/default.css | 551 +++++++++++++++++++++++ apidocs/css/highlight.css | 97 ++++ apidocs/flow.html | 24 + apidocs/index.html | 3 + apidocs/intro.html | 4 + apidocs/js/highlight.min.js | 2 + apidocs/js/jquery.min.js | 4 + apidocs/js/page_effects.js | 112 +++++ 11 files changed, 1347 insertions(+) create mode 100644 apidocs/clojure.core.async.flow.html create mode 100644 apidocs/clojure.core.async.flow.spi.html create mode 100644 apidocs/clojure.core.async.html create mode 100644 apidocs/css/default.css create mode 100644 apidocs/css/highlight.css create mode 100644 apidocs/flow.html create mode 100644 apidocs/index.html create mode 100644 apidocs/intro.html create mode 100644 apidocs/js/highlight.min.js create mode 100644 apidocs/js/jquery.min.js create mode 100644 apidocs/js/page_effects.js diff --git a/apidocs/clojure.core.async.flow.html b/apidocs/clojure.core.async.flow.html new file mode 100644 index 0000000..d7d4900 --- /dev/null +++ b/apidocs/clojure.core.async.flow.html @@ -0,0 +1,206 @@ + +clojure.core.async.flow documentation

clojure.core.async.flow

+Note - Alpha, work-in-progress, names and other details are in flux 
+
+A library for building concurrent, event driven data processing
+flows out of communication-free functions, while centralizing
+control, reporting, execution and error handling. Built on core.async.
+
+The top-level construct is the flow, comprising:
+a set of processes (generally, threads) - concurrent activities
+a set of channels flowing data into and out of the processes
+a set of channels for centralized control, reporting, error-handling,
+  and execution of the processes
+
+A flow is constructed from flow configuration data which defines a
+directed graph of processes and the connections between
+them. Processes describe their I/O requirements and the
+flow (library) itself creates channels and passes them to the
+processes that requested them. See 'create-flow' for the
+details. The flow configuration provides a centralized place for
+policy decisions regarding process settings, threading, buffering etc.
+
+It is expected that applications will rarely define instances of the
+process protocol but instead use the API functions here, 'process'
+and 'step-process', that implement the process protocol in terms of
+calls to ordinary functions that might include no communication or
+core.async code. In this way the library helps you achieve a strict
+separation of your application logic from its execution,
+communication, lifecycle, error handling and monitoring.
+
+Note that at several points the library calls upon the user to
+supply ids for processes, inputs, outputs etc. These should be
+keywords. When a namespaced keyword is required it is explicitly
+stated. This documentation refers to various keywords utilized by
+the library itself as ::flow/xyz, where ::flow is an alias for
+clojure.core.async.flow
+
+A process is represented in the flow definition by an implementation
+of spi/ProcLauncher that starts it. See the spi docs for
+details.

command-proc

(command-proc g pid cmd-id more-kvs)
synchronously sends a process-specific command with the given id and
+additional kvs to the process. The cmd-id must be ns-qualified with
+a ns you own.

create-flow

(create-flow config)
Creates a flow from the supplied configuration: a map containing the
+keys :procs and :conns, and optionally :mixed-exec/:io-exec/:compute-exec
+
+:procs - a map of pid->proc-def
+where proc-def is a map with keys :proc, :args, :chan-opts
+
+:proc - a function that starts a process
+:args - a map of param->val which will be passed to the process ctor
+:chan-opts - a map of in-or-out-id->{:keys [buf-or-n xform]}, where buf-or-n
+             and xform have their meanings per core.async/chan
+             the default is {:buf-or-n 10}
+
+:conns - a collection of [[from-pid outid] [to-pid inid]] tuples.
+
+Inputs and outputs support multiple connections. When an output is
+connected multiple times every connection will get every message,
+as per a core.async/mult.
+
+:mixed-exec/:io-exec/:compute-exec -> ExecutorService
+These can be used to specify the ExecutorService to use for the
+corresonding workload, in lieu of the lib defaults.
+
+N.B. The flow is not started. See 'start'

futurize

(futurize f & {:keys [exec], :or {exec :mixed}, :as opts})
Takes a fn f and returns a fn that takes the same arguments as f
+and immediately returns a future, having started a thread for the
+indicated workload, or via the supplied executor, that invokes f
+with those args and completes that future with its return.
+
+futurize accepts kwarg options:
+:exec - one of the workloads :mixed, :io, :compute
+        or a j.u.c.ExecutorService object,
+        default :mixed

inject

(inject g [pid io-id :as coord] msgs)
synchronously puts the messages on the channel corresponding to the
+input or output of the process

lift*->step

(lift*->step f)
given a fn f taking one arg and returning a collection of non-nil
+values, create a 'step' fn as needed by step-process, with one input
+and one output (named :in and :out), and no state.

lift1->step

(lift1->step f)
like lift*->step except taking a fn returning one value, which, when
+nil, will yield no output.

pause

(pause g)
pauses a running flow
+

pause-proc

(pause-proc g pid)
pauses a process
+

ping

(ping g)
pings all processes, which will put their status and state on the
+report channel returned from start

ping-proc

(ping-proc g pid)
pings the process, which will put its status and state on the report
+channel returned from start

process

(process process-impl-map)(process process-impl-map {:keys [workload timeout-ms], :or {timeout-ms 5000}, :as opts})
Given a map of functions (described below), returns a launcher that
+creates a process compliant with the process protocol (see the
+spi/ProcLauncher doc). The possible entries for process-impl-map
+are :describe, :init, :transition, :transform and :introduce. This is
+the core facility for defining the logic for processes via ordinary
+functions.
+
+:describe - required, () -> desc
+where desc is a map with keys :params :ins and :outs, each of which
+in turn is a map of keyword to doc string, and :workload with
+possible values of :mixed :io :compute. All entries in the describe
+return map are optional.
+
+:params describes the initial arguments to setup the state for the function.
+:ins enumerates the input[s], for which the flow will create channels
+:outs enumerates the output[s], for which the flow may create channels.
+:workload - describes the nature of the workload, one of :mixed :io or :compute
+        an :io workload should not do extended computation
+        a :compute workload should never block
+
+No key may be present in both :ins and :outs, allowing for a uniform
+channel coordinate system of [:process-id :channel-id]. The
+ins/outs/params returned will be the ins/outs/params of the
+process. describe may be called by users to understand how to use
+the proc. It will also be called by the impl in order to discover
+what channels are needed.
+
+:init - optional, (arg-map) -> initial-state
+
+init will be called once by the process to establish any
+initial state. The arg-map will be a map of param->val, as supplied
+in the flow def. init must be provided if 'describe' returns :params.
+
+:transition - optional, (state transition) -> state'
+
+transition will be called when the process makes a state transition,
+transition being one of ::flow/resume, ::flow/pause or ::flow/stop
+
+With this fn a process impl can track changes and coordinate
+resources, especially cleaning up any resources on :stop, since the
+process will no longer be used following that. See the SPI for
+details. state' will be the state supplied to subsequent calls.
+
+Exactly one of either :transform or :introduce are required.
+
+:transform - (state in-name msg) -> [state' output]
+where output is a map of outid->[msgs*]
+
+The transform fn will be called every time a message arrives at any
+of the inputs. Output can be sent to none, any or all of the :outs
+enumerated, and/or an input named by a [pid inid] tuple (e.g. for
+reply-to), and/or to the ::flow/report output. A step need not
+output at all (output or msgs can be empyt/nil), however an output _message_
+may never be nil (per core.async channels). state' will be the state
+supplied to subsequent calls.
+
+:introduce - (state) -> [state' output]
+where output is a map of outid->[msgs*], per :transform
+
+The introduce fn is used for sources - proc-impls that introduce new data
+into the flow by doing I/O with something external to the flow and
+feeding that data to its outputs. A proc-impl specifying :introduce may not
+specify any :ins in its descriptor, as none but the ::flow/control channel
+will be read. Instead, introduce will be called every time through the
+process loop, and will presumably do blocking or paced I/O to get
+new data to return via its outputs. If it does blocking I/O it
+should do so with a timeout so it can regularly return to the
+process loop which can then look for control messages - it's fine
+for introduce to return with no output. Do not spin poll in the introduce
+fn.
+
+process accepts an option map with keys:
+:workload - one of :mixed, :io or :compute
+:compute-timeout-ms - if :workload is :compute, this timeout (default 5000 msec)
+              will be used when getting the return from the future - see below
+
+A :workload supplied as an option to process will override
+any :workload returned by the :describe fn of the process. If neither
+are provded the default is :mixed.
+
+The :compute workload is not allowed for proc impls that
+provide :introduce (as I/O is presumed).
+
+In the :workload context of :mixed or :io, this dictates the type of
+thread in which the process loop will run, _including its calls to
+transform/introduce_. 
+
+When :io is specified transform/introduce should not do extensive computation.
+
+When :compute is specified (only allowed for :transform), each call
+to transform will be run in a separate thread. The process loop will
+run in an :io context (since it no longer directly calls transform,
+all it does is I/O) and it will submit transform to the :compute
+executor then await (blocking, for compute-timeout-ms) the
+completion of the future returned by the executor. If the future
+times out it will be reported on ::flow/error.
+
+When :compute is specified transform must not block!

resume

(resume g)
resumes a paused flow
+

resume-proc

(resume-proc g pid)
resumes a process
+

start

(start g)
starts the entire flow from init values. The processes start paused.
+Call 'resume' or 'resume-proc' to start flow.  returns a map with keys:
+
+:report-chan - a core.async chan for reading.'ping' reponses
+will show up here, as will any explicit ::flow/report outputs
+from :transform/:introduce
+
+:error-chan - a core.async chan for reading. Any (and only)
+exceptions thrown anywhere on any thread inside a flow will appear
+in maps sent here. There will at least be a ::flow/ex entry with the
+exception, and may be additional keys for pid, state, status etc
+depending on the context of the error.

step-process

(step-process f)(step-process f opts)
Given a (e.g. communication-free) step function f of three
+arities (described below), and the same opts as 'process', returns a
+launcher that creates a process compliant with the process
+protocol (see 'process').
+
+The arities of f are:
+
+()->desc
+a function matching the semantics of process' :describe
+
+(arg-map)->initial-state
+a function matching the semantics of process' :init
+
+(state in-name msg)->[state' output]
+a function matching the semantics of process' :transform

stop

(stop g)
shuts down the flow, stopping all procsesses and closing the error
+and report channels. The flow can be started again
\ No newline at end of file diff --git a/apidocs/clojure.core.async.flow.spi.html b/apidocs/clojure.core.async.flow.spi.html new file mode 100644 index 0000000..e9048a6 --- /dev/null +++ b/apidocs/clojure.core.async.flow.spi.html @@ -0,0 +1,74 @@ + +clojure.core.async.flow.spi documentation

clojure.core.async.flow.spi

ProcLauncher

protocol

Note - defining a ProcLauncher is an advanced feature and should not
+be needed for ordinary use of the library. This protocol is for
+creating new types of Processes that are not possible to create
+with ::flow/process.
+
+A ProcLauncher is a constructor for a process, a thread of activity.
+It has two functions - to describe the parameters and input/output
+requirements of the process and to start it. The launcher should
+acquire no resources, nor retain any connection to the started
+process. A launcher may be called upon to start a process more than
+once, and should start a new process each time start is called.
+
+The process launched process must obey the following:
+
+It must have 2 logical statuses, :paused and :running. In
+the :paused status operation is suspended and no output is
+produced.
+
+When the process starts it must be :paused
+
+Whenever it is reading or writing to any channel a process must use
+alts!! and include a read of the ::flow/control channel, giving it
+priority.
+
+Command messages sent over the ::flow/control channel have the keys:
+::flow/to - either ::flow/all or a process id
+::flow/command - ::flow/stop|pause|resume|ping or process-specific
+
+It must act upon any, and only, control messages whose ::flow/to key is its pid or ::flow/all
+It must act upon the following values of ::flow/command:
+
+::flow/stop - all resources should be cleaned up and any thread(s)
+   should exit ordinarily - there will be no more subsequent use
+   of the process.
+::flow/pause - enter the :paused status
+::flow/resume - enter the :running status and resume processing
+::flow/ping - emit a ping message (format TBD) to
+   the ::flow/report channel containing at least its pid and status
+
+A process can define and respond to other commands in its own namespace.
+
+A process should not transmit channel objects (use [pid io-id] data
+coordinates instead) A process should not close channels
+
+Finally, if a process encounters an error it must report it on the
+::flow/error channel (format TBD) and attempt to continue, though it
+may subsequently get a ::flow/stop command it must respect

members

describe

(describe p)
returns a map with keys - :params, :ins and :outs,
+each of which in turn is a map of keyword to docstring
+
+:params describes the initial arguments to setup the state for the process
+:ins enumerates the input[s], for which the graph will create channels
+:outs enumerates the output[s], for which the graph may create channels.
+
+describe may be called by users to understand how to use the
+proc. It will also be called by the impl in order to discover what
+channels are needed.

start

(start p {:keys [pid args ins outs resolver]})
return ignored, called for the
+effect of starting the process (typically, starting its thread)
+
+where:
+
+:pid - the id of the process in the graph, so that e.g. it can refer to itself in control, reporting etc
+:args - a map of param->val,  as supplied in the graph def
+:ins - a map of in-id->readable-channel, plus the ::flow/control channel
+:outs - a map of out-id->writeable-channel, plus the ::flow/error and ::flow/report channels
+        N.B. outputs may be nil if not connected
+:resolver - an impl of spi/Resolver, which can be used to find
+            channels given their logical [pid cid] coordinates, as well as to
+            obtain ExecutorServices corresponding to the
+            logical :mixed/:io/:compute contexts

Resolver

protocol

members

get-exec

(get-exec _ context)
returns the ExecutorService for the given context, one
+of :mixed, :io, :compute

get-write-chan

(get-write-chan _ coord)
Given a tuple of [pid cid], returns a core.async chan to
+write to or nil (in which case the output should be dropped,
+e.g. nothing is connected).
\ No newline at end of file diff --git a/apidocs/clojure.core.async.html b/apidocs/clojure.core.async.html new file mode 100644 index 0000000..64fc4ef --- /dev/null +++ b/apidocs/clojure.core.async.html @@ -0,0 +1,270 @@ + +clojure.core.async documentation

clojure.core.async

Facilities for async programming and communication.
+
+go blocks are dispatched over an internal thread pool, which
+defaults to 8 threads. The size of this pool can be modified using
+the Java system property `clojure.core.async.pool-size`.
+
+Set Java system property `clojure.core.async.go-checking` to true
+to validate go blocks do not invoke core.async blocking operations.
+Property is read once, at namespace load time. Recommended for use
+primarily during development. Invalid blocking calls will throw in
+go block threads - use Thread.setDefaultUncaughtExceptionHandler()
+to catch and handle.

<!

(<! port)
takes a val from port. Must be called inside a (go ...) block. Will
+return nil if closed. Will park if nothing is available.

<!!

(<!! port)
takes a val from port. Will return nil if closed. Will block
+if nothing is available.
+Not intended for use in direct or transitive calls from (go ...) blocks.
+Use the clojure.core.async.go-checking flag to detect invalid use (see
+namespace docs).

>!

(>! port val)
puts a val into port. nil values are not allowed. Must be called
+inside a (go ...) block. Will park if no buffer space is available.
+Returns true unless port is already closed.

>!!

(>!! port val)
puts a val into port. nil values are not allowed. Will block if no
+buffer space is available. Returns true unless port is already closed.
+Not intended for use in direct or transitive calls from (go ...) blocks.
+Use the clojure.core.async.go-checking flag to detect invalid use (see
+namespace docs).

admix

(admix mix ch)
Adds ch as an input to the mix
+

alt!

macro

(alt! & clauses)
Makes a single choice between one of several channel operations,
+as if by alts!, returning the value of the result expr corresponding
+to the operation completed. Must be called inside a (go ...) block.
+
+Each clause takes the form of:
+
+channel-op[s] result-expr
+
+where channel-ops is one of:
+
+take-port - a single port to take
+[take-port | [put-port put-val] ...] - a vector of ports as per alts!
+:default | :priority - an option for alts!
+
+and result-expr is either a list beginning with a vector, whereupon that
+vector will be treated as a binding for the [val port] return of the
+operation, else any other expression.
+
+(alt!
+  [c t] ([val ch] (foo ch val))
+  x ([v] v)
+  [[out val]] :wrote
+  :default 42)
+
+Each option may appear at most once. The choice and parking
+characteristics are those of alts!.

alt!!

macro

(alt!! & clauses)
Like alt!, except as if by alts!!, will block until completed, and
+not intended for use in (go ...) blocks.

alts!

(alts! ports & {:as opts})
Completes at most one of several channel operations. Must be called
+inside a (go ...) block. ports is a vector of channel endpoints,
+which can be either a channel to take from or a vector of
+[channel-to-put-to val-to-put], in any combination. Takes will be
+made as if by <!, and puts will be made as if by >!. Unless
+the :priority option is true, if more than one port operation is
+ready a non-deterministic choice will be made. If no operation is
+ready and a :default value is supplied, [default-val :default] will
+be returned, otherwise alts! will park until the first operation to
+become ready completes. Returns [val port] of the completed
+operation, where val is the value taken for takes, and a
+boolean (true unless already closed, as per put!) for puts.
+
+opts are passed as :key val ... Supported options:
+
+:default val - the value to use if none of the operations are immediately ready
+:priority true - (default nil) when true, the operations will be tried in order.
+
+Note: there is no guarantee that the port exps or val exprs will be
+used, nor in what order should they be, so they should not be
+depended upon for side effects.

alts!!

(alts!! ports & opts)
Like alts!, except takes will be made as if by <!!, and puts will
+be made as if by >!!, will block until completed.
+Not intended for use in direct or transitive calls from (go ...) blocks.
+Use the clojure.core.async.go-checking flag to detect invalid use (see
+namespace docs).

buffer

(buffer n)
Returns a fixed buffer of size n. When full, puts will block/park.
+

chan

(chan)(chan buf-or-n)(chan buf-or-n xform)(chan buf-or-n xform ex-handler)
Creates a channel with an optional buffer, an optional transducer
+(like (map f), (filter p) etc or a composition thereof), and an
+optional exception-handler.  If buf-or-n is a number, will create
+and use a fixed buffer of that size. If a transducer is supplied a
+buffer must be specified. ex-handler must be a fn of one argument -
+if an exception occurs during transformation it will be called with
+the Throwable as an argument, and any non-nil return value will be
+placed in the channel.

close!

(close! chan)
Closes a channel. The channel will no longer accept any puts (they
+will be ignored). Data in the channel remains available for taking, until
+exhausted, after which takes will return nil. If there are any
+pending takes, they will be dispatched with nil. Closing a closed
+channel is a no-op. Returns nil.
+
+Logically closing happens after all puts have been delivered. Therefore, any
+blocked or parked puts will remain blocked/parked until a taker releases them.

defblockingop

macro

(defblockingop op doc arglist & body)

do-alt

(do-alt alts clauses)

do-alts

(do-alts fret ports opts)
returns derefable [val port] if immediate, nil if enqueued
+

dropping-buffer

(dropping-buffer n)
Returns a buffer of size n. When full, puts will complete but
+val will be dropped (no transfer).

fn-handler

(fn-handler f)(fn-handler f blockable)

go

macro

(go & body)
Asynchronously executes the body, returning immediately to the
+calling thread. Additionally, any visible calls to <!, >! and alt!/alts!
+channel operations within the body will block (if necessary) by
+'parking' the calling thread rather than tying up an OS thread (or
+the only JS thread when in ClojureScript). Upon completion of the
+operation, the body will be resumed.
+
+go blocks should not (either directly or indirectly) perform operations
+that may block indefinitely. Doing so risks depleting the fixed pool of
+go block threads, causing all go block processing to stop. This includes
+core.async blocking ops (those ending in !!) and other blocking IO.
+
+Returns a channel which will receive the result of the body when
+completed

go-loop

macro

(go-loop bindings & body)
Like (go (loop ...))
+

into

(into coll ch)
Returns a channel containing the single (collection) result of the
+items taken from the channel conjoined to the supplied
+collection. ch must close before into produces a result.

ioc-alts!

(ioc-alts! state cont-block ports & {:as opts})

map

(map f chs)(map f chs buf-or-n)
Takes a function and a collection of source channels, and returns a
+channel which contains the values produced by applying f to the set
+of first items taken from each source channel, followed by applying
+f to the set of second items from each channel, until any one of the
+channels is closed, at which point the output channel will be
+closed. The returned channel will be unbuffered by default, or a
+buf-or-n can be supplied

merge

(merge chs)(merge chs buf-or-n)
Takes a collection of source channels and returns a channel which
+contains all values taken from them. The returned channel will be
+unbuffered by default, or a buf-or-n can be supplied. The channel
+will close after all the source channels have closed.

Mix

protocol

members

admix*

(admix* m ch)

solo-mode*

(solo-mode* m mode)

toggle*

(toggle* m state-map)

unmix*

(unmix* m ch)

unmix-all*

(unmix-all* m)

mix

(mix out)
Creates and returns a mix of one or more input channels which will
+be put on the supplied out channel. Input sources can be added to
+the mix with 'admix', and removed with 'unmix'. A mix supports
+soloing, muting and pausing multiple inputs atomically using
+'toggle', and can solo using either muting or pausing as determined
+by 'solo-mode'.
+
+Each channel can have zero or more boolean modes set via 'toggle':
+
+:solo - when true, only this (ond other soloed) channel(s) will appear
+        in the mix output channel. :mute and :pause states of soloed
+        channels are ignored. If solo-mode is :mute, non-soloed
+        channels are muted, if :pause, non-soloed channels are
+        paused.
+
+:mute - muted channels will have their contents consumed but not included in the mix
+:pause - paused channels will not have their contents consumed (and thus also not included in the mix)

Mult

protocol

members

tap*

(tap* m ch close?)

untap*

(untap* m ch)

untap-all*

(untap-all* m)

mult

(mult ch)
Creates and returns a mult(iple) of the supplied channel. Channels
+containing copies of the channel can be created with 'tap', and
+detached with 'untap'.
+
+Each item is distributed to all taps in parallel and synchronously,
+i.e. each tap must accept before the next item is distributed. Use
+buffering/windowing to prevent slow taps from holding up the mult.
+
+Items received when there are no taps get dropped.
+
+If a tap puts to a closed channel, it will be removed from the mult.

Mux

protocol

members

muxch*

(muxch* _)

offer!

(offer! port val)
Puts a val into port if it's possible to do so immediately.
+nil values are not allowed. Never blocks. Returns true if offer succeeds.

onto-chan

deprecated in 1.2

(onto-chan ch coll)(onto-chan ch coll close?)
Deprecated - use onto-chan! or onto-chan!!
+

onto-chan!

(onto-chan! ch coll)(onto-chan! ch coll close?)
Puts the contents of coll into the supplied channel.
+
+By default the channel will be closed after the items are copied,
+but can be determined by the close? parameter.
+
+Returns a channel which will close after the items are copied.
+
+If accessing coll might block, use onto-chan!! instead

onto-chan!!

(onto-chan!! ch coll)(onto-chan!! ch coll close?)
Like onto-chan! for use when accessing coll might block,
+e.g. a lazy seq of blocking operations

pipe

(pipe from to)(pipe from to close?)
Takes elements from the from channel and supplies them to the to
+channel. By default, the to channel will be closed when the from
+channel closes, but can be determined by the close?  parameter. Will
+stop consuming the from channel if the to channel closes

pipeline

(pipeline n to xf from)(pipeline n to xf from close?)(pipeline n to xf from close? ex-handler)
Takes elements from the from channel and supplies them to the to
+channel, subject to the transducer xf, with parallelism n. Because
+it is parallel, the transducer will be applied independently to each
+element, not across elements, and may produce zero or more outputs
+per input.  Outputs will be returned in order relative to the
+inputs. By default, the to channel will be closed when the from
+channel closes, but can be determined by the close?  parameter. Will
+stop consuming the from channel if the to channel closes. Note this
+should be used for computational parallelism. If you have multiple
+blocking operations to put in flight, use pipeline-blocking instead,
+If you have multiple asynchronous operations to put in flight, use
+pipeline-async instead. See chan for semantics of ex-handler.

pipeline-async

(pipeline-async n to af from)(pipeline-async n to af from close?)
Takes elements from the from channel and supplies them to the to
+channel, subject to the async function af, with parallelism n. af
+must be a function of two arguments, the first an input value and
+the second a channel on which to place the result(s). The
+presumption is that af will return immediately, having launched some
+asynchronous operation whose completion/callback will put results on
+the channel, then close! it. Outputs will be returned in order
+relative to the inputs. By default, the to channel will be closed
+when the from channel closes, but can be determined by the close?
+parameter. Will stop consuming the from channel if the to channel
+closes. See also pipeline, pipeline-blocking.

pipeline-blocking

(pipeline-blocking n to xf from)(pipeline-blocking n to xf from close?)(pipeline-blocking n to xf from close? ex-handler)
Like pipeline, for blocking operations.
+

poll!

(poll! port)
Takes a val from port if it's possible to do so immediately.
+Never blocks. Returns value if successful, nil otherwise.

promise-chan

(promise-chan)(promise-chan xform)(promise-chan xform ex-handler)
Creates a promise channel with an optional transducer, and an optional
+exception-handler. A promise channel can take exactly one value that consumers
+will receive. Once full, puts complete but val is dropped (no transfer).
+Consumers will block until either a value is placed in the channel or the
+channel is closed, then return the value (or nil) forever. See chan for the
+semantics of xform and ex-handler.

Pub

protocol

members

sub*

(sub* p v ch close?)

unsub*

(unsub* p v ch)

unsub-all*

(unsub-all* p)(unsub-all* p v)

pub

(pub ch topic-fn)(pub ch topic-fn buf-fn)
Creates and returns a pub(lication) of the supplied channel,
+partitioned into topics by the topic-fn. topic-fn will be applied to
+each value on the channel and the result will determine the 'topic'
+on which that value will be put. Channels can be subscribed to
+receive copies of topics using 'sub', and unsubscribed using
+'unsub'. Each topic will be handled by an internal mult on a
+dedicated channel. By default these internal channels are
+unbuffered, but a buf-fn can be supplied which, given a topic,
+creates a buffer with desired properties.
+
+Each item is distributed to all subs in parallel and synchronously,
+i.e. each sub must accept before the next item is distributed. Use
+buffering/windowing to prevent slow subs from holding up the pub.
+
+Items received when there are no matching subs get dropped.
+
+Note that if buf-fns are used then each topic is handled
+asynchronously, i.e. if a channel is subscribed to more than one
+topic it should not expect them to be interleaved identically with
+the source.

put!

(put! port val)(put! port val fn1)(put! port val fn1 on-caller?)
Asynchronously puts a val into port, calling fn1 (if supplied) when
+complete, passing false iff port is already closed. nil values are
+not allowed. If on-caller? (default true) is true, and the put is
+immediately accepted, will call fn1 on calling thread.
+
+fn1 may be run in a fixed-size dispatch thread pool and should not
+perform blocking IO, including core.async blocking ops (those that
+end in !!).
+
+Returns true unless port is already closed.

reduce

(reduce f init ch)
f should be a function of 2 arguments. Returns a channel containing
+the single result of applying f to init and the first item from the
+channel, then applying f to that result and the 2nd item, etc. If
+the channel closes without yielding items, returns init and f is not
+called. ch must close before reduce produces a result.

sliding-buffer

(sliding-buffer n)
Returns a buffer of size n. When full, puts will complete, and be
+buffered, but oldest elements in buffer will be dropped (not
+transferred).

solo-mode

(solo-mode mix mode)
Sets the solo mode of the mix. mode must be one of :mute or :pause
+

split

(split p ch)(split p ch t-buf-or-n f-buf-or-n)
Takes a predicate and a source channel and returns a vector of two
+channels, the first of which will contain the values for which the
+predicate returned true, the second those for which it returned
+false.
+
+The out channels will be unbuffered by default, or two buf-or-ns can
+be supplied. The channels will close after the source channel has
+closed.

sub

(sub p topic ch)(sub p topic ch close?)
Subscribes a channel to a topic of a pub.
+
+By default the channel will be closed when the source closes,
+but can be determined by the close? parameter.

take

(take n ch)(take n ch buf-or-n)
Returns a channel that will return, at most, n items from ch. After n items
+ have been returned, or ch has been closed, the return channel will close.
+
+The output channel is unbuffered by default, unless buf-or-n is given.

take!

(take! port fn1)(take! port fn1 on-caller?)
Asynchronously takes a val from port, passing to fn1. Will pass nil
+if closed. If on-caller? (default true) is true, and value is
+immediately available, will call fn1 on calling thread.
+
+fn1 may be run in a fixed-size dispatch thread pool and should not
+perform blocking IO, including core.async blocking ops (those that
+end in !!).
+
+Returns nil.

tap

(tap mult ch)(tap mult ch close?)
Copies the mult source onto the supplied channel.
+
+By default the channel will be closed when the source closes,
+but can be determined by the close? parameter.

thread

macro

(thread & body)
Executes the body in another thread, returning immediately to the
+calling thread. Returns a channel which will receive the result of
+the body when completed, then close.

thread-call

(thread-call f)
Executes f in another thread, returning immediately to the calling
+thread. Returns a channel which will receive the result of calling
+f when completed, then close.

timeout

(timeout msecs)
Returns a channel that will close after msecs
+

to-chan

deprecated in 1.2

(to-chan coll)
Deprecated - use to-chan! or to-chan!!
+

to-chan!

(to-chan! coll)
Creates and returns a channel which contains the contents of coll,
+closing when exhausted.
+
+If accessing coll might block, use to-chan!! instead

to-chan!!

(to-chan!! coll)
Like to-chan! for use when accessing coll might block,
+e.g. a lazy seq of blocking operations

toggle

(toggle mix state-map)
Atomically sets the state(s) of one or more channels in a mix. The
+state map is a map of channels -> channel-state-map. A
+channel-state-map is a map of attrs -> boolean, where attr is one or
+more of :mute, :pause or :solo. Any states supplied are merged with
+the current state.
+
+Note that channels can be added to a mix via toggle, which can be
+used to add channels in a particular (e.g. paused) state.

transduce

(transduce xform f init ch)
async/reduces a channel with a transformation (xform f).
+Returns a channel containing the result.  ch must close before
+transduce produces a result.

unblocking-buffer?

(unblocking-buffer? buff)
Returns true if a channel created with buff will never block. That is to say,
+puts into this buffer will never cause the buffer to be full. 

unmix

(unmix mix ch)
Removes ch as an input to the mix
+

unmix-all

(unmix-all mix)
removes all inputs from the mix
+

unsub

(unsub p topic ch)
Unsubscribes a channel from a topic of a pub
+

unsub-all

(unsub-all p)(unsub-all p topic)
Unsubscribes all channels from a pub, or a topic of a pub
+

untap

(untap mult ch)
Disconnects a target channel from a mult
+

untap-all

(untap-all mult)
Disconnects all target channels from a mult
+
\ No newline at end of file diff --git a/apidocs/css/default.css b/apidocs/css/default.css new file mode 100644 index 0000000..33f78fe --- /dev/null +++ b/apidocs/css/default.css @@ -0,0 +1,551 @@ +body { + font-family: Helvetica, Arial, sans-serif; + font-size: 15px; +} + +pre, code { + font-family: Monaco, DejaVu Sans Mono, Consolas, monospace; + font-size: 9pt; + margin: 15px 0; +} + +h1 { + font-weight: normal; + font-size: 29px; + margin: 10px 0 2px 0; + padding: 0; +} + +h2 { + font-weight: normal; + font-size: 25px; +} + +h5.license { + margin: 9px 0 22px 0; + color: #555; + font-weight: normal; + font-size: 12px; + font-style: italic; +} + +.document h1, .namespace-index h1 { + font-size: 32px; + margin-top: 12px; +} + +#header, #content, .sidebar { + position: fixed; +} + +#header { + top: 0; + left: 0; + right: 0; + height: 22px; + color: #f5f5f5; + padding: 5px 7px; +} + +#content { + top: 32px; + right: 0; + bottom: 0; + overflow: auto; + background: #fff; + color: #333; + padding: 0 18px; +} + +.sidebar { + position: fixed; + top: 32px; + bottom: 0; + overflow: auto; +} + +.sidebar.primary { + background: #e2e2e2; + border-right: solid 1px #cccccc; + left: 0; + width: 250px; +} + +.sidebar.secondary { + background: #f2f2f2; + border-right: solid 1px #d7d7d7; + left: 251px; + width: 200px; +} + +#content.namespace-index, #content.document { + left: 251px; +} + +#content.namespace-docs { + left: 452px; +} + +#content.document { + padding-bottom: 10%; +} + +#header { + background: #3f3f3f; + box-shadow: 0 0 8px rgba(0, 0, 0, 0.4); + z-index: 100; +} + +#header h1 { + margin: 0; + padding: 0; + font-size: 18px; + font-weight: lighter; + text-shadow: -1px -1px 0px #333; +} + +#header h1 .project-version { + font-weight: normal; +} + +.project-version { + padding-left: 0.15em; +} + +#header a, .sidebar a { + display: block; + text-decoration: none; +} + +#header a { + color: #f5f5f5; +} + +.sidebar a { + color: #333; +} + +#header h2 { + float: right; + font-size: 9pt; + font-weight: normal; + margin: 4px 3px; + padding: 0; + color: #bbb; +} + +#header h2 a { + display: inline; +} + +.sidebar h3 { + margin: 0; + padding: 10px 13px 0 13px; + font-size: 19px; + font-weight: lighter; +} + +.sidebar h3 a { + color: #444; +} + +.sidebar h3.no-link { + color: #636363; +} + +.sidebar ul { + padding: 7px 0 6px 0; + margin: 0; +} + +.sidebar ul.index-link { + padding-bottom: 4px; +} + +.sidebar li { + display: block; + vertical-align: middle; +} + +.sidebar li a, .sidebar li .no-link { + border-left: 3px solid transparent; + padding: 0 10px; + white-space: nowrap; +} + +.sidebar li .no-link { + display: block; + color: #777; + font-style: italic; +} + +.sidebar li .inner { + display: inline-block; + padding-top: 7px; + height: 24px; +} + +.sidebar li a, .sidebar li .tree { + height: 31px; +} + +.depth-1 .inner { padding-left: 2px; } +.depth-2 .inner { padding-left: 6px; } +.depth-3 .inner { padding-left: 20px; } +.depth-4 .inner { padding-left: 34px; } +.depth-5 .inner { padding-left: 48px; } +.depth-6 .inner { padding-left: 62px; } + +.sidebar li .tree { + display: block; + float: left; + position: relative; + top: -10px; + margin: 0 4px 0 0; + padding: 0; +} + +.sidebar li.depth-1 .tree { + display: none; +} + +.sidebar li .tree .top, .sidebar li .tree .bottom { + display: block; + margin: 0; + padding: 0; + width: 7px; +} + +.sidebar li .tree .top { + border-left: 1px solid #aaa; + border-bottom: 1px solid #aaa; + height: 19px; +} + +.sidebar li .tree .bottom { + height: 22px; +} + +.sidebar li.branch .tree .bottom { + border-left: 1px solid #aaa; +} + +.sidebar.primary li.current a { + border-left: 3px solid #a33; + color: #a33; +} + +.sidebar.secondary li.current a { + border-left: 3px solid #33a; + color: #33a; +} + +.namespace-index h2 { + margin: 30px 0 0 0; +} + +.namespace-index h3 { + font-size: 16px; + font-weight: bold; + margin-bottom: 0; +} + +.namespace-index .topics { + padding-left: 30px; + margin: 11px 0 0 0; +} + +.namespace-index .topics li { + padding: 5px 0; +} + +.namespace-docs h3 { + font-size: 18px; + font-weight: bold; +} + +.public h3 { + margin: 0; + float: left; +} + +.usage { + clear: both; +} + +.public { + margin: 0; + border-top: 1px solid #e0e0e0; + padding-top: 14px; + padding-bottom: 6px; +} + +.public:last-child { + margin-bottom: 20%; +} + +.members .public:last-child { + margin-bottom: 0; +} + +.members { + margin: 15px 0; +} + +.members h4 { + color: #555; + font-weight: normal; + font-variant: small-caps; + margin: 0 0 5px 0; +} + +.members .inner { + padding-top: 5px; + padding-left: 12px; + margin-top: 2px; + margin-left: 7px; + border-left: 1px solid #bbb; +} + +#content .members .inner h3 { + font-size: 12pt; +} + +.members .public { + border-top: none; + margin-top: 0; + padding-top: 6px; + padding-bottom: 0; +} + +.members .public:first-child { + padding-top: 0; +} + +h4.type, +h4.dynamic, +h4.added, +h4.deprecated { + float: left; + margin: 3px 10px 15px 0; + font-size: 15px; + font-weight: bold; + font-variant: small-caps; +} + +.public h4.type, +.public h4.dynamic, +.public h4.added, +.public h4.deprecated { + font-size: 13px; + font-weight: bold; + margin: 3px 0 0 10px; +} + +.members h4.type, +.members h4.added, +.members h4.deprecated { + margin-top: 1px; +} + +h4.type { + color: #717171; +} + +h4.dynamic { + color: #9933aa; +} + +h4.added { + color: #508820; +} + +h4.deprecated { + color: #880000; +} + +.namespace { + margin-bottom: 30px; +} + +.namespace:last-child { + margin-bottom: 10%; +} + +.index { + padding: 0; + font-size: 80%; + margin: 15px 0; + line-height: 16px; +} + +.index * { + display: inline; +} + +.index p { + padding-right: 3px; +} + +.index li { + padding-right: 5px; +} + +.index ul { + padding-left: 0; +} + +.type-sig { + clear: both; + color: #088; +} + +.type-sig pre { + padding-top: 10px; + margin: 0; +} + +.usage code { + display: block; + color: #008; + margin: 2px 0; +} + +.usage code:first-child { + padding-top: 10px; +} + +p { + margin: 15px 0; +} + +.public p:first-child, .public pre.plaintext { + margin-top: 12px; +} + +.doc { + margin: 0 0 26px 0; + clear: both; +} + +.public .doc { + margin: 0; +} + +.namespace-index .doc { + margin-bottom: 20px; +} + +.namespace-index .namespace .doc { + margin-bottom: 10px; +} + +.markdown p, .markdown li, .markdown dt, .markdown dd, .markdown td { + line-height: 22px; +} + +.markdown li { + padding: 2px 0; +} + +.markdown h2 { + font-weight: normal; + font-size: 25px; + margin: 30px 0 10px 0; +} + +.markdown h3 { + font-weight: normal; + font-size: 20px; + margin: 30px 0 0 0; +} + +.markdown h4 { + font-size: 15px; + margin: 22px 0 -4px 0; +} + +.doc, .public, .namespace .index { + max-width: 680px; + overflow-x: visible; +} + +.markdown pre > code { + display: block; + padding: 10px; +} + +.markdown pre > code, .src-link a { + border: 1px solid #e4e4e4; + border-radius: 2px; +} + +.markdown code:not(.hljs), .src-link a { + background: #f6f6f6; +} + +pre.deps { + display: inline-block; + margin: 0 10px; + border: 1px solid #e4e4e4; + border-radius: 2px; + padding: 10px; + background-color: #f6f6f6; +} + +.markdown hr { + border-style: solid; + border-top: none; + color: #ccc; +} + +.doc ul, .doc ol { + padding-left: 30px; +} + +.doc table { + border-collapse: collapse; + margin: 0 10px; +} + +.doc table td, .doc table th { + border: 1px solid #dddddd; + padding: 4px 6px; +} + +.doc table th { + background: #f2f2f2; +} + +.doc dl { + margin: 0 10px 20px 10px; +} + +.doc dl dt { + font-weight: bold; + margin: 0; + padding: 3px 0; + border-bottom: 1px solid #ddd; +} + +.doc dl dd { + padding: 5px 0; + margin: 0 0 5px 10px; +} + +.doc abbr { + border-bottom: 1px dotted #333; + font-variant: none; + cursor: help; +} + +.src-link { + margin-bottom: 15px; +} + +.src-link a { + font-size: 70%; + padding: 1px 4px; + text-decoration: none; + color: #5555bb; +} diff --git a/apidocs/css/highlight.css b/apidocs/css/highlight.css new file mode 100644 index 0000000..d0cdaa3 --- /dev/null +++ b/apidocs/css/highlight.css @@ -0,0 +1,97 @@ +/* +github.com style (c) Vasily Polovnyov +*/ + +.hljs { + display: block; + overflow-x: auto; + padding: 0.5em; + color: #333; + background: #f8f8f8; +} + +.hljs-comment, +.hljs-quote { + color: #998; + font-style: italic; +} + +.hljs-keyword, +.hljs-selector-tag, +.hljs-subst { + color: #333; + font-weight: bold; +} + +.hljs-number, +.hljs-literal, +.hljs-variable, +.hljs-template-variable, +.hljs-tag .hljs-attr { + color: #008080; +} + +.hljs-string, +.hljs-doctag { + color: #d14; +} + +.hljs-title, +.hljs-section, +.hljs-selector-id { + color: #900; + font-weight: bold; +} + +.hljs-subst { + font-weight: normal; +} + +.hljs-type, +.hljs-class .hljs-title { + color: #458; + font-weight: bold; +} + +.hljs-tag, +.hljs-name, +.hljs-attribute { + color: #000080; + font-weight: normal; +} + +.hljs-regexp, +.hljs-link { + color: #009926; +} + +.hljs-symbol, +.hljs-bullet { + color: #990073; +} + +.hljs-built_in, +.hljs-builtin-name { + color: #0086b3; +} + +.hljs-meta { + color: #999; + font-weight: bold; +} + +.hljs-deletion { + background: #fdd; +} + +.hljs-addition { + background: #dfd; +} + +.hljs-emphasis { + font-style: italic; +} + +.hljs-strong { + font-weight: bold; +} diff --git a/apidocs/flow.html b/apidocs/flow.html new file mode 100644 index 0000000..33a5136 --- /dev/null +++ b/apidocs/flow.html @@ -0,0 +1,24 @@ + +core.async.flow #

core.async.flow

+

Rationale

+

The rationale for core.async says “There comes a time in all good programs when components or subsystems must stop communicating directly with one another.” And core.async provides fundamental tools (channels) for doing that.

+

But using core.async well, e.g. keeping your I/O out of your computational logic, requires discipline and architectural savvy, and to do so consistently throughout an application or organization, conventions. Given channels, many architectural decisions remain regarding thread execution, backpressure, error handling etc. And often the topology of your network of communicating processes emerges out of the flow of control of your program as various pieces of code create threads and wire channels together, interleaved with computation, making it difficult to see the topology or administer it in one place.

+

The fundamental objective of core.async.flow is to enable a strict separation of your application logic from its topology, execution, communication, lifecycle, monitoring and error handling, all of which are provided by and centralized in, c.a.flow, yielding more consistent, robust, testable, observable and operable systems.

+

Overview

+

core.async.flow provides concrete implementations of two more abstractions - the ‘process’ - a thread of activity, and the ‘flow’ - a directed graph of processes communicating via channels. A single data structure describes your flow topology, and has all of the settings for threads, channels etc. A process accepts data from and provides data to channels. The process implementation in c.a.flow handles all channel I/O, thread lifecycle and coordination with the flow graph.

+

All you need to do in you application is:

+
    +
  1. +

    Define ordinary, often pure, data->data functions that the processes will run in their inner loop to do the computational part of processing messages. These functions do not handle channels or threads or lifecycle, and do not even know they are running in a flow. They can be tested in isolation, and hot-reloaded. If they encounter a problem they can, and should, just throw an exception. The process will take care of it from there.

    +
  2. +
  3. +

    Define a flow by creating a data structure that enumerates the processes and the connections between their inputs and outputs, as well as various configuration settings for both.

    +
  4. +
+

With these application inputs, c.a.flow does the rest. It inquires of the processes what channels they require, creates those channels, then instantiates the processes making all of the channel connections between them. The processes in turn start threads (in fully user-configurable thread pools), await inputs, monitor the admin control channel, and when inputs arrive make data->data calls to your application logic, taking the return from that and sending it to the designated output channels. The processes follow a protocol used by the flow to do lifecycle management and error handling.

+

Once you’ve created a flow, the API provides functions to start/stop(shutdown) the flow, and to pause/resume both the flow and individual processes, to ping processes to get their state and that of their connected channels, to inject data into any point in the graph etc. The flow provides channels containing the ordinary monitoring/reporting stream and, separately, the error stream.

+

The library provides many more details and features, including the ability to create, via ordinary functions, processes that act as sources (of data from outside the flow) or sinks (to recipients outside the flow) so you can situate your flow in a broader context while still coordinating resource management with the flow lifecycle.

+

I hope core.async.flow enables you to write more robust and smaller applications, with more focus on your domain logic and less on plumbing.

+

Rich Hickey 1/2025

+
\ No newline at end of file diff --git a/apidocs/index.html b/apidocs/index.html new file mode 100644 index 0000000..b7367fc --- /dev/null +++ b/apidocs/index.html @@ -0,0 +1,3 @@ + + \ No newline at end of file diff --git a/apidocs/intro.html b/apidocs/intro.html new file mode 100644 index 0000000..290d669 --- /dev/null +++ b/apidocs/intro.html @@ -0,0 +1,4 @@ + +Introduction to core.async

Introduction to core.async

+
\ No newline at end of file diff --git a/apidocs/js/highlight.min.js b/apidocs/js/highlight.min.js new file mode 100644 index 0000000..6486ffd --- /dev/null +++ b/apidocs/js/highlight.min.js @@ -0,0 +1,2 @@ +/*! highlight.js v9.6.0 | BSD3 License | git.io/hljslicense */ +!function(e){var n="object"==typeof window&&window||"object"==typeof self&&self;"undefined"!=typeof exports?e(exports):n&&(n.hljs=e({}),"function"==typeof define&&define.amd&&define([],function(){return n.hljs}))}(function(e){function n(e){return e.replace(/[&<>]/gm,function(e){return I[e]})}function t(e){return e.nodeName.toLowerCase()}function r(e,n){var t=e&&e.exec(n);return t&&0===t.index}function a(e){return k.test(e)}function i(e){var n,t,r,i,o=e.className+" ";if(o+=e.parentNode?e.parentNode.className:"",t=B.exec(o))return R(t[1])?t[1]:"no-highlight";for(o=o.split(/\s+/),n=0,r=o.length;r>n;n++)if(i=o[n],a(i)||R(i))return i}function o(e,n){var t,r={};for(t in e)r[t]=e[t];if(n)for(t in n)r[t]=n[t];return r}function u(e){var n=[];return function r(e,a){for(var i=e.firstChild;i;i=i.nextSibling)3===i.nodeType?a+=i.nodeValue.length:1===i.nodeType&&(n.push({event:"start",offset:a,node:i}),a=r(i,a),t(i).match(/br|hr|img|input/)||n.push({event:"stop",offset:a,node:i}));return a}(e,0),n}function c(e,r,a){function i(){return e.length&&r.length?e[0].offset!==r[0].offset?e[0].offset"}function u(e){l+=""}function c(e){("start"===e.event?o:u)(e.node)}for(var s=0,l="",f=[];e.length||r.length;){var g=i();if(l+=n(a.substr(s,g[0].offset-s)),s=g[0].offset,g===e){f.reverse().forEach(u);do c(g.splice(0,1)[0]),g=i();while(g===e&&g.length&&g[0].offset===s);f.reverse().forEach(o)}else"start"===g[0].event?f.push(g[0].node):f.pop(),c(g.splice(0,1)[0])}return l+n(a.substr(s))}function s(e){function n(e){return e&&e.source||e}function t(t,r){return new RegExp(n(t),"m"+(e.cI?"i":"")+(r?"g":""))}function r(a,i){if(!a.compiled){if(a.compiled=!0,a.k=a.k||a.bK,a.k){var u={},c=function(n,t){e.cI&&(t=t.toLowerCase()),t.split(" ").forEach(function(e){var t=e.split("|");u[t[0]]=[n,t[1]?Number(t[1]):1]})};"string"==typeof a.k?c("keyword",a.k):E(a.k).forEach(function(e){c(e,a.k[e])}),a.k=u}a.lR=t(a.l||/\w+/,!0),i&&(a.bK&&(a.b="\\b("+a.bK.split(" ").join("|")+")\\b"),a.b||(a.b=/\B|\b/),a.bR=t(a.b),a.e||a.eW||(a.e=/\B|\b/),a.e&&(a.eR=t(a.e)),a.tE=n(a.e)||"",a.eW&&i.tE&&(a.tE+=(a.e?"|":"")+i.tE)),a.i&&(a.iR=t(a.i)),null==a.r&&(a.r=1),a.c||(a.c=[]);var s=[];a.c.forEach(function(e){e.v?e.v.forEach(function(n){s.push(o(e,n))}):s.push("self"===e?a:e)}),a.c=s,a.c.forEach(function(e){r(e,a)}),a.starts&&r(a.starts,i);var l=a.c.map(function(e){return e.bK?"\\.?("+e.b+")\\.?":e.b}).concat([a.tE,a.i]).map(n).filter(Boolean);a.t=l.length?t(l.join("|"),!0):{exec:function(){return null}}}}r(e)}function l(e,t,a,i){function o(e,n){var t,a;for(t=0,a=n.c.length;a>t;t++)if(r(n.c[t].bR,e))return n.c[t]}function u(e,n){if(r(e.eR,n)){for(;e.endsParent&&e.parent;)e=e.parent;return e}return e.eW?u(e.parent,n):void 0}function c(e,n){return!a&&r(n.iR,e)}function g(e,n){var t=N.cI?n[0].toLowerCase():n[0];return e.k.hasOwnProperty(t)&&e.k[t]}function h(e,n,t,r){var a=r?"":y.classPrefix,i='',i+n+o}function p(){var e,t,r,a;if(!E.k)return n(B);for(a="",t=0,E.lR.lastIndex=0,r=E.lR.exec(B);r;)a+=n(B.substr(t,r.index-t)),e=g(E,r),e?(M+=e[1],a+=h(e[0],n(r[0]))):a+=n(r[0]),t=E.lR.lastIndex,r=E.lR.exec(B);return a+n(B.substr(t))}function d(){var e="string"==typeof E.sL;if(e&&!x[E.sL])return n(B);var t=e?l(E.sL,B,!0,L[E.sL]):f(B,E.sL.length?E.sL:void 0);return E.r>0&&(M+=t.r),e&&(L[E.sL]=t.top),h(t.language,t.value,!1,!0)}function b(){k+=null!=E.sL?d():p(),B=""}function v(e){k+=e.cN?h(e.cN,"",!0):"",E=Object.create(e,{parent:{value:E}})}function m(e,n){if(B+=e,null==n)return b(),0;var t=o(n,E);if(t)return t.skip?B+=n:(t.eB&&(B+=n),b(),t.rB||t.eB||(B=n)),v(t,n),t.rB?0:n.length;var r=u(E,n);if(r){var a=E;a.skip?B+=n:(a.rE||a.eE||(B+=n),b(),a.eE&&(B=n));do E.cN&&(k+=C),E.skip||(M+=E.r),E=E.parent;while(E!==r.parent);return r.starts&&v(r.starts,""),a.rE?0:n.length}if(c(n,E))throw new Error('Illegal lexeme "'+n+'" for mode "'+(E.cN||"")+'"');return B+=n,n.length||1}var N=R(e);if(!N)throw new Error('Unknown language: "'+e+'"');s(N);var w,E=i||N,L={},k="";for(w=E;w!==N;w=w.parent)w.cN&&(k=h(w.cN,"",!0)+k);var B="",M=0;try{for(var I,j,O=0;;){if(E.t.lastIndex=O,I=E.t.exec(t),!I)break;j=m(t.substr(O,I.index-O),I[0]),O=I.index+j}for(m(t.substr(O)),w=E;w.parent;w=w.parent)w.cN&&(k+=C);return{r:M,value:k,language:e,top:E}}catch(T){if(T.message&&-1!==T.message.indexOf("Illegal"))return{r:0,value:n(t)};throw T}}function f(e,t){t=t||y.languages||E(x);var r={r:0,value:n(e)},a=r;return t.filter(R).forEach(function(n){var t=l(n,e,!1);t.language=n,t.r>a.r&&(a=t),t.r>r.r&&(a=r,r=t)}),a.language&&(r.second_best=a),r}function g(e){return y.tabReplace||y.useBR?e.replace(M,function(e,n){return y.useBR&&"\n"===e?"
":y.tabReplace?n.replace(/\t/g,y.tabReplace):void 0}):e}function h(e,n,t){var r=n?L[n]:t,a=[e.trim()];return e.match(/\bhljs\b/)||a.push("hljs"),-1===e.indexOf(r)&&a.push(r),a.join(" ").trim()}function p(e){var n,t,r,o,s,p=i(e);a(p)||(y.useBR?(n=document.createElementNS("http://www.w3.org/1999/xhtml","div"),n.innerHTML=e.innerHTML.replace(/\n/g,"").replace(//g,"\n")):n=e,s=n.textContent,r=p?l(p,s,!0):f(s),t=u(n),t.length&&(o=document.createElementNS("http://www.w3.org/1999/xhtml","div"),o.innerHTML=r.value,r.value=c(t,u(o),s)),r.value=g(r.value),e.innerHTML=r.value,e.className=h(e.className,p,r.language),e.result={language:r.language,re:r.r},r.second_best&&(e.second_best={language:r.second_best.language,re:r.second_best.r}))}function d(e){y=o(y,e)}function b(){if(!b.called){b.called=!0;var e=document.querySelectorAll("pre code");w.forEach.call(e,p)}}function v(){addEventListener("DOMContentLoaded",b,!1),addEventListener("load",b,!1)}function m(n,t){var r=x[n]=t(e);r.aliases&&r.aliases.forEach(function(e){L[e]=n})}function N(){return E(x)}function R(e){return e=(e||"").toLowerCase(),x[e]||x[L[e]]}var w=[],E=Object.keys,x={},L={},k=/^(no-?highlight|plain|text)$/i,B=/\blang(?:uage)?-([\w-]+)\b/i,M=/((^(<[^>]+>|\t|)+|(?:\n)))/gm,C="
",y={classPrefix:"hljs-",tabReplace:null,useBR:!1,languages:void 0},I={"&":"&","<":"<",">":">"};return e.highlight=l,e.highlightAuto=f,e.fixMarkup=g,e.highlightBlock=p,e.configure=d,e.initHighlighting=b,e.initHighlightingOnLoad=v,e.registerLanguage=m,e.listLanguages=N,e.getLanguage=R,e.inherit=o,e.IR="[a-zA-Z]\\w*",e.UIR="[a-zA-Z_]\\w*",e.NR="\\b\\d+(\\.\\d+)?",e.CNR="(-?)(\\b0[xX][a-fA-F0-9]+|(\\b\\d+(\\.\\d*)?|\\.\\d+)([eE][-+]?\\d+)?)",e.BNR="\\b(0b[01]+)",e.RSR="!|!=|!==|%|%=|&|&&|&=|\\*|\\*=|\\+|\\+=|,|-|-=|/=|/|:|;|<<|<<=|<=|<|===|==|=|>>>=|>>=|>=|>>>|>>|>|\\?|\\[|\\{|\\(|\\^|\\^=|\\||\\|=|\\|\\||~",e.BE={b:"\\\\[\\s\\S]",r:0},e.ASM={cN:"string",b:"'",e:"'",i:"\\n",c:[e.BE]},e.QSM={cN:"string",b:'"',e:'"',i:"\\n",c:[e.BE]},e.PWM={b:/\b(a|an|the|are|I'm|isn't|don't|doesn't|won't|but|just|should|pretty|simply|enough|gonna|going|wtf|so|such|will|you|your|like)\b/},e.C=function(n,t,r){var a=e.inherit({cN:"comment",b:n,e:t,c:[]},r||{});return a.c.push(e.PWM),a.c.push({cN:"doctag",b:"(?:TODO|FIXME|NOTE|BUG|XXX):",r:0}),a},e.CLCM=e.C("//","$"),e.CBCM=e.C("/\\*","\\*/"),e.HCM=e.C("#","$"),e.NM={cN:"number",b:e.NR,r:0},e.CNM={cN:"number",b:e.CNR,r:0},e.BNM={cN:"number",b:e.BNR,r:0},e.CSSNM={cN:"number",b:e.NR+"(%|em|ex|ch|rem|vw|vh|vmin|vmax|cm|mm|in|pt|pc|px|deg|grad|rad|turn|s|ms|Hz|kHz|dpi|dpcm|dppx)?",r:0},e.RM={cN:"regexp",b:/\//,e:/\/[gimuy]*/,i:/\n/,c:[e.BE,{b:/\[/,e:/\]/,r:0,c:[e.BE]}]},e.TM={cN:"title",b:e.IR,r:0},e.UTM={cN:"title",b:e.UIR,r:0},e.METHOD_GUARD={b:"\\.\\s*"+e.UIR,r:0},e});hljs.registerLanguage("clojure",function(e){var t={"builtin-name":"def defonce cond apply if-not if-let if not not= = < > <= >= == + / * - rem quot neg? pos? delay? symbol? keyword? true? false? integer? empty? coll? list? set? ifn? fn? associative? sequential? sorted? counted? reversible? number? decimal? class? distinct? isa? float? rational? reduced? ratio? odd? even? char? seq? vector? string? map? nil? contains? zero? instance? not-every? not-any? libspec? -> ->> .. . inc compare do dotimes mapcat take remove take-while drop letfn drop-last take-last drop-while while intern condp case reduced cycle split-at split-with repeat replicate iterate range merge zipmap declare line-seq sort comparator sort-by dorun doall nthnext nthrest partition eval doseq await await-for let agent atom send send-off release-pending-sends add-watch mapv filterv remove-watch agent-error restart-agent set-error-handler error-handler set-error-mode! error-mode shutdown-agents quote var fn loop recur throw try monitor-enter monitor-exit defmacro defn defn- macroexpand macroexpand-1 for dosync and or when when-not when-let comp juxt partial sequence memoize constantly complement identity assert peek pop doto proxy defstruct first rest cons defprotocol cast coll deftype defrecord last butlast sigs reify second ffirst fnext nfirst nnext defmulti defmethod meta with-meta ns in-ns create-ns import refer keys select-keys vals key val rseq name namespace promise into transient persistent! conj! assoc! dissoc! pop! disj! use class type num float double short byte boolean bigint biginteger bigdec print-method print-dup throw-if printf format load compile get-in update-in pr pr-on newline flush read slurp read-line subvec with-open memfn time re-find re-groups rand-int rand mod locking assert-valid-fdecl alias resolve ref deref refset swap! reset! set-validator! compare-and-set! alter-meta! reset-meta! commute get-validator alter ref-set ref-history-count ref-min-history ref-max-history ensure sync io! new next conj set! to-array future future-call into-array aset gen-class reduce map filter find empty hash-map hash-set sorted-map sorted-map-by sorted-set sorted-set-by vec vector seq flatten reverse assoc dissoc list disj get union difference intersection extend extend-type extend-protocol int nth delay count concat chunk chunk-buffer chunk-append chunk-first chunk-rest max min dec unchecked-inc-int unchecked-inc unchecked-dec-inc unchecked-dec unchecked-negate unchecked-add-int unchecked-add unchecked-subtract-int unchecked-subtract chunk-next chunk-cons chunked-seq? prn vary-meta lazy-seq spread list* str find-keyword keyword symbol gensym force rationalize"},r="a-zA-Z_\\-!.?+*=<>&#'",n="["+r+"]["+r+"0-9/;:]*",a="[-+]?\\d+(\\.\\d+)?",o={b:n,r:0},s={cN:"number",b:a,r:0},i=e.inherit(e.QSM,{i:null}),c=e.C(";","$",{r:0}),d={cN:"literal",b:/\b(true|false|nil)\b/},l={b:"[\\[\\{]",e:"[\\]\\}]"},m={cN:"comment",b:"\\^"+n},p=e.C("\\^\\{","\\}"),u={cN:"symbol",b:"[:]{1,2}"+n},f={b:"\\(",e:"\\)"},h={eW:!0,r:0},y={k:t,l:n,cN:"name",b:n,starts:h},b=[f,i,m,p,c,u,l,s,d,o];return f.c=[e.C("comment",""),y,h],h.c=b,l.c=b,{aliases:["clj"],i:/\S/,c:[f,i,m,p,c,u,l,s,d]}});hljs.registerLanguage("clojure-repl",function(e){return{c:[{cN:"meta",b:/^([\w.-]+|\s*#_)=>/,starts:{e:/$/,sL:"clojure"}}]}}); \ No newline at end of file diff --git a/apidocs/js/jquery.min.js b/apidocs/js/jquery.min.js new file mode 100644 index 0000000..73f33fb --- /dev/null +++ b/apidocs/js/jquery.min.js @@ -0,0 +1,4 @@ +/*! jQuery v1.11.0 | (c) 2005, 2014 jQuery Foundation, Inc. | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=c.slice,e=c.concat,f=c.push,g=c.indexOf,h={},i=h.toString,j=h.hasOwnProperty,k="".trim,l={},m="1.11.0",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return d.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return n.each(this,a,b)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(d.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor(null)},push:f,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return a-parseFloat(a)>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!j.call(a,"constructor")&&!j.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(l.ownLast)for(b in a)return j.call(a,b);for(b in a);return void 0===b||j.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?h[i.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b,c){var d,e=0,f=a.length,g=s(a);if(c){if(g){for(;f>e;e++)if(d=b.apply(a[e],c),d===!1)break}else for(e in a)if(d=b.apply(a[e],c),d===!1)break}else if(g){for(;f>e;e++)if(d=b.call(a[e],e,a[e]),d===!1)break}else for(e in a)if(d=b.call(a[e],e,a[e]),d===!1)break;return a},trim:k&&!k.call("\ufeff\xa0")?function(a){return null==a?"":k.call(a)}:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):f.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(g)return g.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,f=0,g=a.length,h=s(a),i=[];if(h)for(;g>f;f++)d=b(a[f],f,c),null!=d&&i.push(d);else for(f in a)d=b(a[f],f,c),null!=d&&i.push(d);return e.apply([],i)},guid:1,proxy:function(a,b){var c,e,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=d.call(arguments,2),e=function(){return a.apply(b||this,c.concat(d.call(arguments)))},e.guid=a.guid=a.guid||n.guid++,e):void 0},now:function(){return+new Date},support:l}),n.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s="sizzle"+-new Date,t=a.document,u=0,v=0,w=eb(),x=eb(),y=eb(),z=function(a,b){return a===b&&(j=!0),0},A="undefined",B=1<<31,C={}.hasOwnProperty,D=[],E=D.pop,F=D.push,G=D.push,H=D.slice,I=D.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},J="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",K="[\\x20\\t\\r\\n\\f]",L="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",M=L.replace("w","w#"),N="\\["+K+"*("+L+")"+K+"*(?:([*^$|!~]?=)"+K+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+M+")|)|)"+K+"*\\]",O=":("+L+")(?:\\(((['\"])((?:\\\\.|[^\\\\])*?)\\3|((?:\\\\.|[^\\\\()[\\]]|"+N.replace(3,8)+")*)|.*)\\)|)",P=new RegExp("^"+K+"+|((?:^|[^\\\\])(?:\\\\.)*)"+K+"+$","g"),Q=new RegExp("^"+K+"*,"+K+"*"),R=new RegExp("^"+K+"*([>+~]|"+K+")"+K+"*"),S=new RegExp("="+K+"*([^\\]'\"]*?)"+K+"*\\]","g"),T=new RegExp(O),U=new RegExp("^"+M+"$"),V={ID:new RegExp("^#("+L+")"),CLASS:new RegExp("^\\.("+L+")"),TAG:new RegExp("^("+L.replace("w","w*")+")"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+K+"*(even|odd|(([+-]|)(\\d*)n|)"+K+"*(?:([+-]|)"+K+"*(\\d+)|))"+K+"*\\)|)","i"),bool:new RegExp("^(?:"+J+")$","i"),needsContext:new RegExp("^"+K+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+K+"*((?:-\\d)?\\d*)"+K+"*\\)|)(?=[^-]|$)","i")},W=/^(?:input|select|textarea|button)$/i,X=/^h\d$/i,Y=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,$=/[+~]/,_=/'|\\/g,ab=new RegExp("\\\\([\\da-f]{1,6}"+K+"?|("+K+")|.)","ig"),bb=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)};try{G.apply(D=H.call(t.childNodes),t.childNodes),D[t.childNodes.length].nodeType}catch(cb){G={apply:D.length?function(a,b){F.apply(a,H.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function db(a,b,d,e){var f,g,h,i,j,m,p,q,u,v;if((b?b.ownerDocument||b:t)!==l&&k(b),b=b||l,d=d||[],!a||"string"!=typeof a)return d;if(1!==(i=b.nodeType)&&9!==i)return[];if(n&&!e){if(f=Z.exec(a))if(h=f[1]){if(9===i){if(g=b.getElementById(h),!g||!g.parentNode)return d;if(g.id===h)return d.push(g),d}else if(b.ownerDocument&&(g=b.ownerDocument.getElementById(h))&&r(b,g)&&g.id===h)return d.push(g),d}else{if(f[2])return G.apply(d,b.getElementsByTagName(a)),d;if((h=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return G.apply(d,b.getElementsByClassName(h)),d}if(c.qsa&&(!o||!o.test(a))){if(q=p=s,u=b,v=9===i&&a,1===i&&"object"!==b.nodeName.toLowerCase()){m=ob(a),(p=b.getAttribute("id"))?q=p.replace(_,"\\$&"):b.setAttribute("id",q),q="[id='"+q+"'] ",j=m.length;while(j--)m[j]=q+pb(m[j]);u=$.test(a)&&mb(b.parentNode)||b,v=m.join(",")}if(v)try{return G.apply(d,u.querySelectorAll(v)),d}catch(w){}finally{p||b.removeAttribute("id")}}}return xb(a.replace(P,"$1"),b,d,e)}function eb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function fb(a){return a[s]=!0,a}function gb(a){var b=l.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function hb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function ib(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||B)-(~a.sourceIndex||B);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function jb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function kb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function lb(a){return fb(function(b){return b=+b,fb(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function mb(a){return a&&typeof a.getElementsByTagName!==A&&a}c=db.support={},f=db.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},k=db.setDocument=function(a){var b,e=a?a.ownerDocument||a:t,g=e.defaultView;return e!==l&&9===e.nodeType&&e.documentElement?(l=e,m=e.documentElement,n=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){k()},!1):g.attachEvent&&g.attachEvent("onunload",function(){k()})),c.attributes=gb(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=gb(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Y.test(e.getElementsByClassName)&&gb(function(a){return a.innerHTML="
",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=gb(function(a){return m.appendChild(a).id=s,!e.getElementsByName||!e.getElementsByName(s).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==A&&n){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ab,bb);return function(a){var c=typeof a.getAttributeNode!==A&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==A?b.getElementsByTagName(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return typeof b.getElementsByClassName!==A&&n?b.getElementsByClassName(a):void 0},p=[],o=[],(c.qsa=Y.test(e.querySelectorAll))&&(gb(function(a){a.innerHTML="",a.querySelectorAll("[t^='']").length&&o.push("[*^$]="+K+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||o.push("\\["+K+"*(?:value|"+J+")"),a.querySelectorAll(":checked").length||o.push(":checked")}),gb(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&o.push("name"+K+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||o.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),o.push(",.*:")})),(c.matchesSelector=Y.test(q=m.webkitMatchesSelector||m.mozMatchesSelector||m.oMatchesSelector||m.msMatchesSelector))&&gb(function(a){c.disconnectedMatch=q.call(a,"div"),q.call(a,"[s!='']:x"),p.push("!=",O)}),o=o.length&&new RegExp(o.join("|")),p=p.length&&new RegExp(p.join("|")),b=Y.test(m.compareDocumentPosition),r=b||Y.test(m.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},z=b?function(a,b){if(a===b)return j=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===e||a.ownerDocument===t&&r(t,a)?-1:b===e||b.ownerDocument===t&&r(t,b)?1:i?I.call(i,a)-I.call(i,b):0:4&d?-1:1)}:function(a,b){if(a===b)return j=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],k=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:i?I.call(i,a)-I.call(i,b):0;if(f===g)return ib(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)k.unshift(c);while(h[d]===k[d])d++;return d?ib(h[d],k[d]):h[d]===t?-1:k[d]===t?1:0},e):l},db.matches=function(a,b){return db(a,null,null,b)},db.matchesSelector=function(a,b){if((a.ownerDocument||a)!==l&&k(a),b=b.replace(S,"='$1']"),!(!c.matchesSelector||!n||p&&p.test(b)||o&&o.test(b)))try{var d=q.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return db(b,l,null,[a]).length>0},db.contains=function(a,b){return(a.ownerDocument||a)!==l&&k(a),r(a,b)},db.attr=function(a,b){(a.ownerDocument||a)!==l&&k(a);var e=d.attrHandle[b.toLowerCase()],f=e&&C.call(d.attrHandle,b.toLowerCase())?e(a,b,!n):void 0;return void 0!==f?f:c.attributes||!n?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},db.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},db.uniqueSort=function(a){var b,d=[],e=0,f=0;if(j=!c.detectDuplicates,i=!c.sortStable&&a.slice(0),a.sort(z),j){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return i=null,a},e=db.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=db.selectors={cacheLength:50,createPseudo:fb,match:V,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ab,bb),a[3]=(a[4]||a[5]||"").replace(ab,bb),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||db.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&db.error(a[0]),a},PSEUDO:function(a){var b,c=!a[5]&&a[2];return V.CHILD.test(a[0])?null:(a[3]&&void 0!==a[4]?a[2]=a[4]:c&&T.test(c)&&(b=ob(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ab,bb).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=w[a+" "];return b||(b=new RegExp("(^|"+K+")"+a+"("+K+"|$)"))&&w(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==A&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=db.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),t=!i&&!h;if(q){if(f){while(p){l=b;while(l=l[p])if(h?l.nodeName.toLowerCase()===r:1===l.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&t){k=q[s]||(q[s]={}),j=k[a]||[],n=j[0]===u&&j[1],m=j[0]===u&&j[2],l=n&&q.childNodes[n];while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if(1===l.nodeType&&++m&&l===b){k[a]=[u,n,m];break}}else if(t&&(j=(b[s]||(b[s]={}))[a])&&j[0]===u)m=j[1];else while(l=++n&&l&&l[p]||(m=n=0)||o.pop())if((h?l.nodeName.toLowerCase()===r:1===l.nodeType)&&++m&&(t&&((l[s]||(l[s]={}))[a]=[u,m]),l===b))break;return m-=e,m===d||m%d===0&&m/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||db.error("unsupported pseudo: "+a);return e[s]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?fb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=I.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:fb(function(a){var b=[],c=[],d=g(a.replace(P,"$1"));return d[s]?fb(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),!c.pop()}}),has:fb(function(a){return function(b){return db(a,b).length>0}}),contains:fb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:fb(function(a){return U.test(a||"")||db.error("unsupported lang: "+a),a=a.replace(ab,bb).toLowerCase(),function(b){var c;do if(c=n?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===m},focus:function(a){return a===l.activeElement&&(!l.hasFocus||l.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return X.test(a.nodeName)},input:function(a){return W.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:lb(function(){return[0]}),last:lb(function(a,b){return[b-1]}),eq:lb(function(a,b,c){return[0>c?c+b:c]}),even:lb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:lb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:lb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:lb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function qb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=v++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j=[u,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(i=b[s]||(b[s]={}),(h=i[d])&&h[0]===u&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function rb(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function sb(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(!c||c(f,d,e))&&(g.push(f),j&&b.push(h));return g}function tb(a,b,c,d,e,f){return d&&!d[s]&&(d=tb(d)),e&&!e[s]&&(e=tb(e,f)),fb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||wb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:sb(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=sb(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?I.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=sb(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):G.apply(g,r)})}function ub(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],i=g||d.relative[" "],j=g?1:0,k=qb(function(a){return a===b},i,!0),l=qb(function(a){return I.call(b,a)>-1},i,!0),m=[function(a,c,d){return!g&&(d||c!==h)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>j;j++)if(c=d.relative[a[j].type])m=[qb(rb(m),c)];else{if(c=d.filter[a[j].type].apply(null,a[j].matches),c[s]){for(e=++j;f>e;e++)if(d.relative[a[e].type])break;return tb(j>1&&rb(m),j>1&&pb(a.slice(0,j-1).concat({value:" "===a[j-2].type?"*":""})).replace(P,"$1"),c,e>j&&ub(a.slice(j,e)),f>e&&ub(a=a.slice(e)),f>e&&pb(a))}m.push(c)}return rb(m)}function vb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,i,j,k){var m,n,o,p=0,q="0",r=f&&[],s=[],t=h,v=f||e&&d.find.TAG("*",k),w=u+=null==t?1:Math.random()||.1,x=v.length;for(k&&(h=g!==l&&g);q!==x&&null!=(m=v[q]);q++){if(e&&m){n=0;while(o=a[n++])if(o(m,g,i)){j.push(m);break}k&&(u=w)}c&&((m=!o&&m)&&p--,f&&r.push(m))}if(p+=q,c&&q!==p){n=0;while(o=b[n++])o(r,s,g,i);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=E.call(j));s=sb(s)}G.apply(j,s),k&&!f&&s.length>0&&p+b.length>1&&db.uniqueSort(j)}return k&&(u=w,h=t),r};return c?fb(f):f}g=db.compile=function(a,b){var c,d=[],e=[],f=y[a+" "];if(!f){b||(b=ob(a)),c=b.length;while(c--)f=ub(b[c]),f[s]?d.push(f):e.push(f);f=y(a,vb(e,d))}return f};function wb(a,b,c){for(var d=0,e=b.length;e>d;d++)db(a,b[d],c);return c}function xb(a,b,e,f){var h,i,j,k,l,m=ob(a);if(!f&&1===m.length){if(i=m[0]=m[0].slice(0),i.length>2&&"ID"===(j=i[0]).type&&c.getById&&9===b.nodeType&&n&&d.relative[i[1].type]){if(b=(d.find.ID(j.matches[0].replace(ab,bb),b)||[])[0],!b)return e;a=a.slice(i.shift().value.length)}h=V.needsContext.test(a)?0:i.length;while(h--){if(j=i[h],d.relative[k=j.type])break;if((l=d.find[k])&&(f=l(j.matches[0].replace(ab,bb),$.test(i[0].type)&&mb(b.parentNode)||b))){if(i.splice(h,1),a=f.length&&pb(i),!a)return G.apply(e,f),e;break}}}return g(a,m)(f,b,!n,e,$.test(a)&&mb(b.parentNode)||b),e}return c.sortStable=s.split("").sort(z).join("")===s,c.detectDuplicates=!!j,k(),c.sortDetached=gb(function(a){return 1&a.compareDocumentPosition(l.createElement("div"))}),gb(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||hb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&gb(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||hb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),gb(function(a){return null==a.getAttribute("disabled")})||hb(J,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),db}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=n.expr.match.needsContext,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^.[^:#\[\.,]*$/;function x(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(w.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>=0!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(x(this,a||[],!1))},not:function(a){return this.pushStack(x(this,a||[],!0))},is:function(a){return!!x(this,"string"==typeof a&&u.test(a)?n(a):a||[],!1).length}});var y,z=a.document,A=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,B=n.fn.init=function(a,b){var c,d;if(!a)return this;if("string"==typeof a){if(c="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:A.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||y).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:z,!0)),v.test(c[1])&&n.isPlainObject(b))for(c in b)n.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=z.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return y.find(a);this.length=1,this[0]=d}return this.context=z,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof y.ready?y.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};B.prototype=n.fn,y=n(z);var C=/^(?:parents|prev(?:Until|All))/,D={children:!0,contents:!0,next:!0,prev:!0};n.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!n(e).is(c)))1===e.nodeType&&d.push(e),e=e[b];return d},sibling:function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c}}),n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=u.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.unique(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.unique(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function E(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return n.dir(a,"parentNode")},parentsUntil:function(a,b,c){return n.dir(a,"parentNode",c)},next:function(a){return E(a,"nextSibling")},prev:function(a){return E(a,"previousSibling")},nextAll:function(a){return n.dir(a,"nextSibling")},prevAll:function(a){return n.dir(a,"previousSibling")},nextUntil:function(a,b,c){return n.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return n.dir(a,"previousSibling",c)},siblings:function(a){return n.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return n.sibling(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(D[a]||(e=n.unique(e)),C.test(a)&&(e=e.reverse())),this.pushStack(e)}});var F=/\S+/g,G={};function H(a){var b=G[a]={};return n.each(a.match(F)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?G[a]||H(a):n.extend({},a);var b,c,d,e,f,g,h=[],i=!a.once&&[],j=function(l){for(c=a.memory&&l,d=!0,f=g||0,g=0,e=h.length,b=!0;h&&e>f;f++)if(h[f].apply(l[0],l[1])===!1&&a.stopOnFalse){c=!1;break}b=!1,h&&(i?i.length&&j(i.shift()):c?h=[]:k.disable())},k={add:function(){if(h){var d=h.length;!function f(b){n.each(b,function(b,c){var d=n.type(c);"function"===d?a.unique&&k.has(c)||h.push(c):c&&c.length&&"string"!==d&&f(c)})}(arguments),b?e=h.length:c&&(g=d,j(c))}return this},remove:function(){return h&&n.each(arguments,function(a,c){var d;while((d=n.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?n.inArray(a,h)>-1:!(!h||!h.length)},empty:function(){return h=[],e=0,this},disable:function(){return h=i=c=void 0,this},disabled:function(){return!h},lock:function(){return i=void 0,c||k.disable(),this},locked:function(){return!i},fireWith:function(a,c){return!h||d&&!i||(c=c||[],c=[a,c.slice?c.slice():c],b?i.push(c):j(c)),this},fire:function(){return k.fireWith(this,arguments),this},fired:function(){return!!d}};return k},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=d.call(arguments),e=c.length,f=1!==e||a&&n.isFunction(a.promise)?e:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(e){b[a]=this,c[a]=arguments.length>1?d.call(arguments):e,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(e>1)for(i=new Array(e),j=new Array(e),k=new Array(e);e>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().done(h(b,k,c)).fail(g.reject).progress(h(b,j,i)):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){if(a===!0?!--n.readyWait:!n.isReady){if(!z.body)return setTimeout(n.ready);n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(z,[n]),n.fn.trigger&&n(z).trigger("ready").off("ready"))}}});function J(){z.addEventListener?(z.removeEventListener("DOMContentLoaded",K,!1),a.removeEventListener("load",K,!1)):(z.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(z.addEventListener||"load"===event.type||"complete"===z.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===z.readyState)setTimeout(n.ready);else if(z.addEventListener)z.addEventListener("DOMContentLoaded",K,!1),a.addEventListener("load",K,!1);else{z.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&z.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!n.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}J(),n.ready()}}()}return I.promise(b)};var L="undefined",M;for(M in n(l))break;l.ownLast="0"!==M,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c=z.getElementsByTagName("body")[0];c&&(a=z.createElement("div"),a.style.cssText="border:0;width:0;height:0;position:absolute;top:0;left:-9999px;margin-top:1px",b=z.createElement("div"),c.appendChild(a).appendChild(b),typeof b.style.zoom!==L&&(b.style.cssText="border:0;margin:0;width:1px;padding:1px;display:inline;zoom:1",(l.inlineBlockNeedsLayout=3===b.offsetWidth)&&(c.style.zoom=1)),c.removeChild(a),a=b=null)}),function(){var a=z.createElement("div");if(null==l.deleteExpando){l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}}a=null}(),n.acceptData=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(n.acceptData(a)){var f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f +}}function S(a,b,c){if(n.acceptData(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d]));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},X=/^(?:checkbox|radio)$/i;!function(){var a=z.createDocumentFragment(),b=z.createElement("div"),c=z.createElement("input");if(b.setAttribute("className","t"),b.innerHTML="
a",l.leadingWhitespace=3===b.firstChild.nodeType,l.tbody=!b.getElementsByTagName("tbody").length,l.htmlSerialize=!!b.getElementsByTagName("link").length,l.html5Clone="<:nav>"!==z.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,a.appendChild(c),l.appendChecked=c.checked,b.innerHTML="",l.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,a.appendChild(b),b.innerHTML="",l.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){l.noCloneEvent=!1}),b.cloneNode(!0).click()),null==l.deleteExpando){l.deleteExpando=!0;try{delete b.test}catch(d){l.deleteExpando=!1}}a=b=c=null}(),function(){var b,c,d=z.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),l[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var Y=/^(?:input|select|textarea)$/i,Z=/^key/,$=/^(?:mouse|contextmenu)|click/,_=/^(?:focusinfocus|focusoutblur)$/,ab=/^([^.]*)(?:\.(.+)|)$/;function bb(){return!0}function cb(){return!1}function db(){try{return z.activeElement}catch(a){}}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof n===L||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(F)||[""],h=b.length;while(h--)f=ab.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.special[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(F)||[""],j=b.length;while(j--)if(h=ab.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.remove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,m,o=[d||z],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||z,3!==d.nodeType&&8!==d.nodeType&&!_.test(p+n.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[n.expando]?b:new n.Event(p,"object"==typeof b&&b),b.isTrigger=e?2:3,b.namespace=q.join("."),b.namespace_re=b.namespace?new RegExp("(^|\\.)"+q.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=d),c=null==c?[b]:n.makeArray(c,[b]),k=n.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!n.isWindow(d)){for(i=k.delegateType||p,_.test(i+p)||(h=h.parentNode);h;h=h.parentNode)o.push(h),l=h;l===(d.ownerDocument||z)&&o.push(l.defaultView||l.parentWindow||a)}m=0;while((h=o[m++])&&!b.isPropagationStopped())b.type=m>1?i:k.bindType||p,f=(n._data(h,"events")||{})[b.type]&&n._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&n.acceptData(h)&&(b.result=f.apply(h,c),b.result===!1&&b.preventDefault());if(b.type=p,!e&&!b.isDefaultPrevented()&&(!k._default||k._default.apply(o.pop(),c)===!1)&&n.acceptData(d)&&g&&d[p]&&!n.isWindow(d)){l=d[g],l&&(d[g]=null),n.event.triggered=p;try{d[p]()}catch(r){}n.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,g=0;while((e=f.handlers[g++])&&!a.isImmediatePropagationStopped())(!a.namespace_re||a.namespace_re.test(e.namespace))&&(a.handleObj=e,a.data=e.data,c=((n.event.special[e.origType]||{}).handle||e.handler).apply(f.elem,i),void 0!==c&&(a.result=c)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&(!a.button||"click"!==a.type))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(e=[],f=0;h>f;f++)d=b[f],c=d.selector+" ",void 0===e[c]&&(e[c]=d.needsContext?n(c,this).index(i)>=0:n.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),ib=/^\s+/,jb=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,kb=/<([\w:]+)/,lb=/\s*$/g,sb={option:[1,""],legend:[1,"
","
"],area:[1,"",""],param:[1,"",""],thead:[1,"","
"],tr:[2,"","
"],col:[2,"","
"],td:[3,"","
"],_default:l.htmlSerialize?[0,"",""]:[1,"X
","
"]},tb=eb(z),ub=tb.appendChild(z.createElement("div"));sb.optgroup=sb.option,sb.tbody=sb.tfoot=sb.colgroup=sb.caption=sb.thead,sb.th=sb.td;function vb(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==L?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==L?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,vb(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function wb(a){X.test(a.type)&&(a.defaultChecked=a.checked)}function xb(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function yb(a){return a.type=(null!==n.find.attr(a,"type"))+"/"+a.type,a}function zb(a){var b=qb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Ab(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}function Bb(a,b){if(1===b.nodeType&&n.hasData(a)){var c,d,e,f=n._data(a),g=n._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)n.event.add(b,c,h[c][d])}g.data&&(g.data=n.extend({},g.data))}}function Cb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!l.noCloneEvent&&b[n.expando]){e=n._data(b);for(d in e.events)n.removeEvent(b,d,e.handle);b.removeAttribute(n.expando)}"script"===c&&b.text!==a.text?(yb(b).text=a.text,zb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),l.html5Clone&&a.innerHTML&&!n.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&X.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:("input"===c||"textarea"===c)&&(b.defaultValue=a.defaultValue)}}n.extend({clone:function(a,b,c){var d,e,f,g,h,i=n.contains(a.ownerDocument,a);if(l.html5Clone||n.isXMLDoc(a)||!hb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(ub.innerHTML=a.outerHTML,ub.removeChild(f=ub.firstChild)),!(l.noCloneEvent&&l.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(d=vb(f),h=vb(a),g=0;null!=(e=h[g]);++g)d[g]&&Cb(e,d[g]);if(b)if(c)for(h=h||vb(a),d=d||vb(f),g=0;null!=(e=h[g]);g++)Bb(e,d[g]);else Bb(a,f);return d=vb(f,"script"),d.length>0&&Ab(d,!i&&vb(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,k,m=a.length,o=eb(b),p=[],q=0;m>q;q++)if(f=a[q],f||0===f)if("object"===n.type(f))n.merge(p,f.nodeType?[f]:f);else if(mb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(kb.exec(f)||["",""])[1].toLowerCase(),k=sb[i]||sb._default,h.innerHTML=k[1]+f.replace(jb,"<$1>")+k[2],e=k[0];while(e--)h=h.lastChild;if(!l.leadingWhitespace&&ib.test(f)&&p.push(b.createTextNode(ib.exec(f)[0])),!l.tbody){f="table"!==i||lb.test(f)?""!==k[1]||lb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)n.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}n.merge(p,h.childNodes),h.textContent="";while(h.firstChild)h.removeChild(h.firstChild);h=o.lastChild}else p.push(b.createTextNode(f));h&&o.removeChild(h),l.appendChecked||n.grep(vb(p,"input"),wb),q=0;while(f=p[q++])if((!d||-1===n.inArray(f,d))&&(g=n.contains(f.ownerDocument,f),h=vb(o.appendChild(f),"script"),g&&Ab(h),c)){e=0;while(f=h[e++])pb.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=n.expando,j=n.cache,k=l.deleteExpando,m=n.event.special;null!=(d=a[h]);h++)if((b||n.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)m[e]?n.event.remove(d,e):n.removeEvent(d,e,g.handle);j[f]&&(delete j[f],k?delete d[i]:typeof d.removeAttribute!==L?d.removeAttribute(i):d[i]=null,c.push(f))}}}),n.fn.extend({text:function(a){return W(this,function(a){return void 0===a?n.text(this):this.empty().append((this[0]&&this[0].ownerDocument||z).createTextNode(a))},null,a,arguments.length)},append:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.appendChild(a)}})},prepend:function(){return this.domManip(arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=xb(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return this.domManip(arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},remove:function(a,b){for(var c,d=a?n.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||n.cleanData(vb(c)),c.parentNode&&(b&&n.contains(c.ownerDocument,c)&&Ab(vb(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&n.cleanData(vb(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&n.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return W(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(gb,""):void 0;if(!("string"!=typeof a||nb.test(a)||!l.htmlSerialize&&hb.test(a)||!l.leadingWhitespace&&ib.test(a)||sb[(kb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(jb,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(vb(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=arguments[0];return this.domManip(arguments,function(b){a=this.parentNode,n.cleanData(vb(this)),a&&a.replaceChild(b,this)}),a&&(a.length||a.nodeType)?this:this.remove()},detach:function(a){return this.remove(a,!0)},domManip:function(a,b){a=e.apply([],a);var c,d,f,g,h,i,j=0,k=this.length,m=this,o=k-1,p=a[0],q=n.isFunction(p);if(q||k>1&&"string"==typeof p&&!l.checkClone&&ob.test(p))return this.each(function(c){var d=m.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(k&&(i=n.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=n.map(vb(i,"script"),yb),f=g.length;k>j;j++)d=i,j!==o&&(d=n.clone(d,!0,!0),f&&n.merge(g,vb(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,n.map(g,zb),j=0;f>j;j++)d=g[j],pb.test(d.type||"")&&!n._data(d,"globalEval")&&n.contains(h,d)&&(d.src?n._evalUrl&&n._evalUrl(d.src):n.globalEval((d.text||d.textContent||d.innerHTML||"").replace(rb,"")));i=c=null}return this}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=0,e=[],g=n(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),n(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Db,Eb={};function Fb(b,c){var d=n(c.createElement(b)).appendTo(c.body),e=a.getDefaultComputedStyle?a.getDefaultComputedStyle(d[0]).display:n.css(d[0],"display");return d.detach(),e}function Gb(a){var b=z,c=Eb[a];return c||(c=Fb(a,b),"none"!==c&&c||(Db=(Db||n("