diff --git a/.gitignore b/.gitignore index 52998c10..85548305 100644 --- a/.gitignore +++ b/.gitignore @@ -1,6 +1,8 @@ *.py[cod] # C extensions +/dpark/portable_hash.c +/dpark/utils/recursion.c *.so # Packages @@ -15,6 +17,8 @@ var sdist develop-eggs .installed.cfg +.eggs +.cache lib lib64 @@ -25,9 +29,13 @@ pip-log.txt .coverage .tox nosetests.xml +.pytest_cache #Translations *.mo #Mr Developer .mr.developer.cfg + +# Pycharm +.idea diff --git a/.travis.yml b/.travis.yml index 4a11d2f6..cb502a3f 100644 --- a/.travis.yml +++ b/.travis.yml @@ -1,16 +1,33 @@ language: python - -python: - - 2.7 - +cache: + directories: + $HOME/.cache/pip +sudo: false +addons: + apt: + packages: + - libzmq3-dev +matrix: + include: + - python: 2.7 + env: TOXENV=py27 + - python: 3.5 + env: TOXENV=py35 + - python: pypy + env: TOXENV=pypy install: - - sudo apt-get update -qq - - sudo apt-get remove libzmq3 - - sudo apt-get install -qq cython libzmq-dev - + - pip install -U setuptools tox script: - - python setup.py test - + - tox branches: - except: - - legacy + except: + - legacy +deploy: + provider: pypi + skip_existing: true + user: windreamer + password: + secure: Qo9ucolE2ajQSbgifbqzfDYD5J09FgYEntXpak1ffPUWxJyeR6H2/uChCGqXC3+ZEkTpzc/MHshHQRRymtPZeHn5etnSHzshWu8njBVLr4I7ft2d5dNGJDxfK2j79HPVnib1MoPNC8kiqP05QD+hyMMQNZDFCHwG1D5bnWNhZYI= + on: + tags: true + repo: douban/dpark diff --git a/CONTRIBUTORS b/CONTRIBUTORS index 817733df..8cc1297c 100644 --- a/CONTRIBUTORS +++ b/CONTRIBUTORS @@ -1,2 +1,3 @@ Scott Smith Tian Zhongbo +Zhu Zhaolong diff --git a/MANIFEST.in b/MANIFEST.in index a1678509..54f4fa84 100644 --- a/MANIFEST.in +++ b/MANIFEST.in @@ -1,3 +1,5 @@ +recursive-include dpark/web/ui/static * +recursive-include dpark/web/ui/templates * include dpark/porable_hash.pyx -include dpark/porable_hash.c -include README.md +include README.rst +include LICENSE diff --git a/README.md b/README.md deleted file mode 100644 index 0d49fb43..00000000 --- a/README.md +++ /dev/null @@ -1,54 +0,0 @@ -# DPark - -[![travis-ci status](https://travis-ci.org/douban/dpark.svg)](https://travis-ci.org/douban/dpark) - -DPark is a Python clone of Spark, MapReduce(R) alike computing -framework supporting iterative computation. - -Example for word counting (`wc.py`): - -``` python - import dpark - file = dpark.textFile("/tmp/words.txt") - words = file.flatMap(lambda x:x.split()).map(lambda x:(x,1)) - wc = words.reduceByKey(lambda x,y:x+y).collectAsMap() - print wc -``` - -This script can run locally or on a Mesos cluster without -any modification, just using different command-line arguments: - -``` bash -$ python wc.py -$ python wc.py -m process -$ python wc.py -m host[:port] -``` - -See examples/ for more use cases. - -Some more docs (in Chinese): https://github.com/jackfengji/test_pro/wiki - -DPark can run with Mesos 0.9 or higher. - -If a `$MESOS_MASTER` environment variable is set, you can use a shortcut and run DPark with Mesos just by typing -``` bash -$ python wc.py -m mesos -``` - -`$MESOS_MASTER` can be any scheme of Mesos master, such as -``` bash -$ export MESOS_MASTER=zk://zk1:2181,zk2:2181,zk3:2181/mesos_master -``` - -In order to speed up shuffling, you should deploy Nginx at port 5055 -for accessing data in `DPARK_WORK_DIR` (default is `/tmp/dpark`), such as: - -``` bash - server { - listen 5055; - server_name localhost; - root /tmp/dpark/; - } -``` - -Mailing list: dpark-users@googlegroups.com (http://groups.google.com/group/dpark-users) diff --git a/README.rst b/README.rst new file mode 100644 index 00000000..8298c7c9 --- /dev/null +++ b/README.rst @@ -0,0 +1,155 @@ +DPark +===== + +|pypi status| |ci status| |gitter| + +DPark is a Python clone of Spark, MapReduce(R) alike computing framework +supporting iterative computation. + +Installation +------------ + +.. code:: bash + + ## Due to the use of C extensions, some libraries need to be installed first. + + $ sudo apt-get install libtool pkg-config build-essential autoconf automake + $ sudo apt-get install python-dev + $ sudo apt-get install libzmq-dev + + ## Then just pip install dpark (``sudo`` maybe needed if you encounter permission problem). + + $ pip install dpark + + +Example +------- + +for word counting (``wc.py``): + +.. code:: python + + from dpark import DparkContext + ctx = DparkContext() + file = ctx.textFile("/tmp/words.txt") + words = file.flatMap(lambda x:x.split()).map(lambda x:(x,1)) + wc = words.reduceByKey(lambda x,y:x+y).collectAsMap() + print wc + +This script can run locally or on a Mesos cluster without any +modification, just using different command-line arguments: + +.. code:: bash + + $ python wc.py + $ python wc.py -m process + $ python wc.py -m host[:port] + +See examples/ for more use cases. + + +Configuration +------------ + +DPark can run with Mesos 0.9 or higher. + +If a ``$MESOS_MASTER`` environment variable is set, you can use a +shortcut and run DPark with Mesos just by typing + +.. code:: bash + + $ python wc.py -m mesos + +``$MESOS_MASTER`` can be any scheme of Mesos master, such as + +.. code:: bash + + $ export MESOS_MASTER=zk://zk1:2181,zk2:2181,zk3:2181/mesos_master + +In order to speed up shuffling, you should deploy Nginx at port 5055 for +accessing data in ``DPARK_WORK_DIR`` (default is ``/tmp/dpark``), such +as: + +.. code:: bash + + server { + listen 5055; + server_name localhost; + root /tmp/dpark/; + } + +UI +-- + +2 DAGs: + +1. stage graph: stage is a running unit, contain a set of task, each run same ops for a split of rdd. +2. use api callsite graph + + +UI when running +~~~~~~~~~~~~~~ + +Just open the url from log like ``start listening on Web UI http://server_01:40812`` . + + +UI after running +~~~~~~~~~~~~~~~~~~ + +1. before run, config LOGHUB & LOGHUB_PATH_FORMAT in dpark.conf, pre-create LOGHUB_DIR. +2. get log hubdir from log like ``logging/prof to LOGHUB_DIR/2018/09/27/16/b2e3349b-9858-4153-b491-80699c757485-8754``, which in clude mesos framework id. +3. run ``dpark_web.py -p 9999 -l LOGHUB_DIR/2018/09/27/16/b2e3349b-9858-4153-b491-80699c757485-8728/``, dpark_web.py is in tools/ + + +UI examples for features +~~~~~~~ + + +show sharing shuffle map output + +.. code:: python + + + rdd = DparkContext().makeRDD([(1,1)]).map(m).groupByKey() + rdd.map(m).collect() + rdd.map(m).collect() + + +.. image:: images/share_mapoutput.png + + +combine nodes iff with same lineage, form a logic tree inside stage, then each node contain a PIPELINE of rdds. + + +.. code:: python + + + rdd1 = get_rdd() + rdd2 = dc.union([get_rdd() for i in range(2)]) + rdd3 = get_rdd().groupByKey() + dc.union([rdd1, rdd2, rdd3]).collect() + + +.. image:: images/unions.png + + +More docs (in Chinese) +------------------------- + +https://dpark.readthedocs.io/zh_CN/latest/ + +https://github.com/jackfengji/test\_pro/wiki + +Mailing list: dpark-users@googlegroups.com +(http://groups.google.com/group/dpark-users) + + +.. |pypi status| image:: https://img.shields.io/pypi/v/DPark.svg + :target: https://pypi.python.org/pypi/DPark + +.. |gitter| image:: https://badges.gitter.im/douban/dpark.svg + :alt: Join the chat at https://gitter.im/douban/dpark + :target: https://gitter.im/douban/dpark?utm_source=badge&utm_medium=badge&utm_campaign=pr-badge&utm_content=badge + +.. |ci status| image:: https://travis-ci.org/douban/dpark.svg + :target: https://travis-ci.org/douban/dpark diff --git a/TODO b/TODO deleted file mode 100644 index 03949514..00000000 --- a/TODO +++ /dev/null @@ -1,3 +0,0 @@ -* run DPark under pypy -* HDFS support -* MFS client support diff --git a/benchmarks/locality.py b/benchmarks/locality.py new file mode 100644 index 00000000..63c93c26 --- /dev/null +++ b/benchmarks/locality.py @@ -0,0 +1,23 @@ +import time +from dpark import DparkContext, optParser +from dpark.file_manager import file_manager +dc = DparkContext() + +optParser.set_usage("%prog [options] path") +options, args = optParser.parse_args() + +path = args[0] + + +def run(split_size=1): + t = time.time() + dc.textFile(path).mergeSplit(splitSize=split_size).filter(lambda x: "yangxiufeng" in x).count() + return time.time() - t + + +run() # file cache +print("{}s with locality".format(run())) +file_manager.fs_list = file_manager.fs_list[1:] +print("{}s merge & without locality".format(run(10))) +print("{}s without locality, ".format(run())) + diff --git a/benchmarks/stream_shuffle.py b/benchmarks/stream_shuffle.py new file mode 100644 index 00000000..cf587495 --- /dev/null +++ b/benchmarks/stream_shuffle.py @@ -0,0 +1,258 @@ +# -*- coding: utf-8 -*- + +from __future__ import absolute_import +from six.moves import range +import time +import os +import gc +import sys +import time +import unittest +import logging + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) + +from dpark import DparkContext +from dpark.utils.nested_groupby import GroupByNestedIter +from dpark.shuffle import GroupByNestedIter, AutoBatchedSerializer +from dpark.utils.profile import profile +import dpark.conf + +GroupByNestedIter.NO_CACHE = True +print_mem_incr = True +print_mem_incr = False +dc = DparkContext('mesos') +RC = dpark.conf.rddconf +M = 1024 * 1024 + + +def rss_func(p): + if hasattr(p, "memory_info"): + mem_info = getattr(p, "memory_info") + else: + mem_info = getattr(p, 'get_memory_info') + + def _(): + return mem_info().rss / M + + return _ + + +class BenchShuffle(object): + + def __init__(self, num_key, num_value_per_key, num_map, num_reduce=1): + self.num_key = num_key + self.num_value_per_key = num_value_per_key + self.num_map = num_map + self.num_reduce = num_reduce + + self.exp_num_key = 0 + self.exp_num_value = 0 + + def gen_data(self, dup_key): + num_key = self.num_key + num_value_per_key = self.num_value_per_key + + def _fm(map_id): + for i in range(num_key): + for j in range(num_value_per_key): + if dup_key: + yield (i, j) + else: + yield ((map_id, i), j) + + self.exp_num_key = self.num_key + if not dup_key: + self.exp_num_key *= self.num_map + + data = range(self.num_map) + data = list(data) + rdd = dc.makeRDD(data, self.num_map) + rdd = rdd.flatMap(_fm) + return rdd + + def count(self, rdd, count_value=False, multi_value=False): + exp_num_key = self.exp_num_key + print_mem_interval = exp_num_key // 100 + + M = 1024 * 1024 + + # @profile() + def _count_time(it): + st = time.time() + nk = 0 + nv = 0 + np = 0 + + import psutil + proc = psutil.Process() + get_rss = rss_func(proc) + + for x in it: + k, v = x + if count_value: + if multi_value: + for vv in v: + for _ in vv: + nv += 1 + else: + for _ in v: + nv += 1 + nk += 1 + + if print_mem_incr and nk % print_mem_interval == 0: + np += 1 + print("%d%%: num=%d, key=%s, %s" % (np, nk, k, get_rss())) + + mm = max_rss() / M + + m0 = get_rss() + gc.collect() + m1 = get_rss() + mp = AutoBatchedSerializer.size_loaded / M + print("Mem(MB) before gc %d, after gc %d, max %d, size_loaed %d" % (m0, m1, mm, mp)) + + ed = time.time() + reduce_time = ed - st + + return [(nk, nv, mm, reduce_time)] + + tst = time.time() + res = rdd.mapPartition(_count_time).collect() + total_time = time.time() - tst + nkey, nvalue, mm, reduce_time = res[0] + + print("reduce/total time = %d/%d, max_rss(M) = %d" % (reduce_time, total_time, mm)) + assert nkey == exp_num_key, (nkey, exp_num_key) + if count_value: + assert nvalue == self.exp_num_value, (nvalue, self.exp_num_value) + + return mm, total_time, reduce_time + + def test_reducebykey(self, rddconf, dup_key=False, taskMemory=None): + rdd = self.gen_data(dup_key=dup_key) + rdd = rdd.reduceByKey(lambda x, y: y, numSplits=self.num_reduce, rddconf=rddconf, taskMemory=taskMemory) + return self.count(rdd) + + def test_groupbykey(self, rddconf, count_value=True, dup_key=True, taskMemory=None): + rdd = self.gen_data(dup_key=dup_key) + rdd = rdd.groupByKey(numSplits=self.num_reduce, rddconf=rddconf, taskMemory=taskMemory) + self.exp_num_value = self.num_key * self.num_value_per_key * self.num_map + return self.count(rdd, count_value) + + def test_cogroup(self, rddconf, count_value=True, dup_key=True, taskMemory=None): + rdd1 = self.gen_data(dup_key=dup_key) + rdd2 = self.gen_data(dup_key=dup_key) + rdd = rdd1.groupWith(rdd2, numSplits=self.num_reduce, rddconf=rddconf, taskMemory=taskMemory) + self.exp_num_value = self.num_map * self.num_key * self.num_value_per_key * 2 + return self.count(rdd, count_value, multi_value=True) + + def test_join(self, rddconf, count_value=True, dup_key=True, taskMemory=None): + rdd1 = self.gen_data(dup_key=dup_key) + rdd2 = BenchShuffle(self.num_key, 2, 2).gen_data(dup_key=dup_key) + rdd = rdd1.join(rdd2, numSplits=self.num_reduce, rddconf=rddconf, taskMemory=taskMemory) + self.exp_num_key = self.exp_num_value = self.num_map * self.num_key * self.num_value_per_key * 2 * 2 + return self.count(rdd) + + +def max_rss(): + import resource + return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss * 1000 + + +# all time is reduce task func run time +class TestShuffle(unittest.TestCase): + + def test_max_open_file(self): + n = dpark.conf.MAX_OPEN_FILE + for num_map in [n, n + 1]: + # should see waring 'fall back to SHUFFLE_DISK' for n+1 + rddconf = dpark.conf.rddconf(sort_merge=True) + BenchShuffle(10, 10, num_map).test_groupbykey(rddconf=rddconf, taskMemory=50) + + def test_oom_many_key(self): + params = [ + (RC(), 2500), # 30 sec + (RC(disk_merge=True), 2500), # 30 sec + (RC(disk_merge=True, dump_mem_ratio=0.6), 1000), # 145 sec + (RC(sort_merge=True), 50), # 43 sec + ] + for rc, m in params: + BenchShuffle(1024 * 128, 2, 100).test_reducebykey(rddconf=rc, dup_key=False, taskMemory=m) + + def test_oom_many_value(self): + params = [ + (RC(), 4000), # 16 sec + (RC(disk_merge=True, dump_mem_ratio=0.7), 1000), # 17 sec + (RC(sort_merge=True), 200), # 16 sec + (RC(sort_merge=True, iter_group=True), 50), # 16 sec + ] + for rc, m in params: + BenchShuffle(1024, 1024, 100).test_groupbykey(rddconf=rc, taskMemory=m) + + def test_oom_onebigkey(self): + params = [ + (RC(), 3500), # 24 sec + (RC(sort_merge=True, iter_group=True), 50), # 15 sec + ] + for rc, m in params: + BenchShuffle(1, 1024 * 1024, 100).test_groupbykey(rddconf=rc, taskMemory=m) + + def test_cogroup(self): + params = [ + (RC(), 7000), # 40 sec + (RC(sort_merge=True, iter_group=True), 50), # 45 sec + ] + for rc, m in params: + BenchShuffle(1, 1024 * 1024, 100).test_cogroup(rddconf=rc, taskMemory=m) + + def test_join(self): + params = [ + (RC(sort_merge=True, iter_group=True), 50), # 208 sec + (RC(), 3500), # 115 sec + ] + for rc, m in params: + BenchShuffle(1, 1024 * 1024, 100).test_join(rddconf=rc, taskMemory=m) + + def test_disk_merge(self): + + values_per_key = 1024 + + def _test(num_key, map_mem, map_disk_merge): + + def mp1(it): + for v in range(values_per_key): + for k in range(num_key): + yield k, v + + def mp2(it): + n = 0 + for k, g in it: + for v in g: + n += 1 + yield n + + rdd0 = dc.makeRDD([0, 1], 2).mapPartition(mp1) + rdd0.mem = map_mem + + rdd1 = rdd0.groupByKey(numSplits=2, rddconf=RC(disk_merge=True, dump_mem_ratio=0.6)).mapPartition(mp2) + res = rdd1.collect() + assert (sum(res) == num_key * values_per_key * 2) + st = dc.scheduler.jobstats[-1] + from pprint import pprint + pprint(st) + + _test(50 * 1024, 100, True) + + +if __name__ == "__main__": + import dpark.conf + + dpark.conf.MULTI_SEGMENT_DUMP = True + rc = dpark.conf.default_rddconf + rc.disk_merge = False + rc.sort_merge = False + rc.iter_group = False + rc.ordered_group = False + rc.dump_mem_ratio = False + unittest.main(verbosity=2) diff --git a/docs/cn/Makefile b/docs/cn/Makefile new file mode 100644 index 00000000..cdcf2e72 --- /dev/null +++ b/docs/cn/Makefile @@ -0,0 +1,216 @@ +# Makefile for Sphinx documentation +# + +# You can set these variables from the command line. +SPHINXOPTS = +SPHINXBUILD = sphinx-build +PAPER = +BUILDDIR = _build + +# User-friendly check for sphinx-build +ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) +$(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) +endif + +# Internal variables. +PAPEROPT_a4 = -D latex_paper_size=a4 +PAPEROPT_letter = -D latex_paper_size=letter +ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . +# the i18n builder cannot share the environment and doctrees with the others +I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . + +.PHONY: help +help: + @echo "Please use \`make ' where is one of" + @echo " html to make standalone HTML files" + @echo " dirhtml to make HTML files named index.html in directories" + @echo " singlehtml to make a single large HTML file" + @echo " pickle to make pickle files" + @echo " json to make JSON files" + @echo " htmlhelp to make HTML files and a HTML help project" + @echo " qthelp to make HTML files and a qthelp project" + @echo " applehelp to make an Apple Help Book" + @echo " devhelp to make HTML files and a Devhelp project" + @echo " epub to make an epub" + @echo " latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter" + @echo " latexpdf to make LaTeX files and run them through pdflatex" + @echo " latexpdfja to make LaTeX files and run them through platex/dvipdfmx" + @echo " text to make text files" + @echo " man to make manual pages" + @echo " texinfo to make Texinfo files" + @echo " info to make Texinfo files and run them through makeinfo" + @echo " gettext to make PO message catalogs" + @echo " changes to make an overview of all changed/added/deprecated items" + @echo " xml to make Docutils-native XML files" + @echo " pseudoxml to make pseudoxml-XML files for display purposes" + @echo " linkcheck to check all external links for integrity" + @echo " doctest to run all doctests embedded in the documentation (if enabled)" + @echo " coverage to run coverage check of the documentation (if enabled)" + +.PHONY: clean +clean: + rm -rf $(BUILDDIR)/* + +.PHONY: html +html: + $(SPHINXBUILD) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/html." + +.PHONY: dirhtml +dirhtml: + $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." + +.PHONY: singlehtml +singlehtml: + $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml + @echo + @echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml." + +.PHONY: pickle +pickle: + $(SPHINXBUILD) -b pickle $(ALLSPHINXOPTS) $(BUILDDIR)/pickle + @echo + @echo "Build finished; now you can process the pickle files." + +.PHONY: json +json: + $(SPHINXBUILD) -b json $(ALLSPHINXOPTS) $(BUILDDIR)/json + @echo + @echo "Build finished; now you can process the JSON files." + +.PHONY: htmlhelp +htmlhelp: + $(SPHINXBUILD) -b htmlhelp $(ALLSPHINXOPTS) $(BUILDDIR)/htmlhelp + @echo + @echo "Build finished; now you can run HTML Help Workshop with the" \ + ".hhp project file in $(BUILDDIR)/htmlhelp." + +.PHONY: qthelp +qthelp: + $(SPHINXBUILD) -b qthelp $(ALLSPHINXOPTS) $(BUILDDIR)/qthelp + @echo + @echo "Build finished; now you can run "qcollectiongenerator" with the" \ + ".qhcp project file in $(BUILDDIR)/qthelp, like this:" + @echo "# qcollectiongenerator $(BUILDDIR)/qthelp/DPark.qhcp" + @echo "To view the help file:" + @echo "# assistant -collectionFile $(BUILDDIR)/qthelp/DPark.qhc" + +.PHONY: applehelp +applehelp: + $(SPHINXBUILD) -b applehelp $(ALLSPHINXOPTS) $(BUILDDIR)/applehelp + @echo + @echo "Build finished. The help book is in $(BUILDDIR)/applehelp." + @echo "N.B. You won't be able to view it unless you put it in" \ + "~/Library/Documentation/Help or install it in your application" \ + "bundle." + +.PHONY: devhelp +devhelp: + $(SPHINXBUILD) -b devhelp $(ALLSPHINXOPTS) $(BUILDDIR)/devhelp + @echo + @echo "Build finished." + @echo "To view the help file:" + @echo "# mkdir -p $$HOME/.local/share/devhelp/DPark" + @echo "# ln -s $(BUILDDIR)/devhelp $$HOME/.local/share/devhelp/DPark" + @echo "# devhelp" + +.PHONY: epub +epub: + $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub + @echo + @echo "Build finished. The epub file is in $(BUILDDIR)/epub." + +.PHONY: latex +latex: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo + @echo "Build finished; the LaTeX files are in $(BUILDDIR)/latex." + @echo "Run \`make' in that directory to run these through (pdf)latex" \ + "(use \`make latexpdf' here to do that automatically)." + +.PHONY: latexpdf +latexpdf: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through pdflatex..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +.PHONY: latexpdfja +latexpdfja: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through platex and dvipdfmx..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf-ja + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +.PHONY: text +text: + $(SPHINXBUILD) -b text $(ALLSPHINXOPTS) $(BUILDDIR)/text + @echo + @echo "Build finished. The text files are in $(BUILDDIR)/text." + +.PHONY: man +man: + $(SPHINXBUILD) -b man $(ALLSPHINXOPTS) $(BUILDDIR)/man + @echo + @echo "Build finished. The manual pages are in $(BUILDDIR)/man." + +.PHONY: texinfo +texinfo: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo + @echo "Build finished. The Texinfo files are in $(BUILDDIR)/texinfo." + @echo "Run \`make' in that directory to run these through makeinfo" \ + "(use \`make info' here to do that automatically)." + +.PHONY: info +info: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo "Running Texinfo files through makeinfo..." + make -C $(BUILDDIR)/texinfo info + @echo "makeinfo finished; the Info files are in $(BUILDDIR)/texinfo." + +.PHONY: gettext +gettext: + $(SPHINXBUILD) -b gettext $(I18NSPHINXOPTS) $(BUILDDIR)/locale + @echo + @echo "Build finished. The message catalogs are in $(BUILDDIR)/locale." + +.PHONY: changes +changes: + $(SPHINXBUILD) -b changes $(ALLSPHINXOPTS) $(BUILDDIR)/changes + @echo + @echo "The overview file is in $(BUILDDIR)/changes." + +.PHONY: linkcheck +linkcheck: + $(SPHINXBUILD) -b linkcheck $(ALLSPHINXOPTS) $(BUILDDIR)/linkcheck + @echo + @echo "Link check complete; look for any errors in the above output " \ + "or in $(BUILDDIR)/linkcheck/output.txt." + +.PHONY: doctest +doctest: + $(SPHINXBUILD) -b doctest $(ALLSPHINXOPTS) $(BUILDDIR)/doctest + @echo "Testing of doctests in the sources finished, look at the " \ + "results in $(BUILDDIR)/doctest/output.txt." + +.PHONY: coverage +coverage: + $(SPHINXBUILD) -b coverage $(ALLSPHINXOPTS) $(BUILDDIR)/coverage + @echo "Testing of coverage in the sources finished, look at the " \ + "results in $(BUILDDIR)/coverage/python.txt." + +.PHONY: xml +xml: + $(SPHINXBUILD) -b xml $(ALLSPHINXOPTS) $(BUILDDIR)/xml + @echo + @echo "Build finished. The XML files are in $(BUILDDIR)/xml." + +.PHONY: pseudoxml +pseudoxml: + $(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml + @echo + @echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml." diff --git a/docs/cn/conf.py b/docs/cn/conf.py new file mode 100644 index 00000000..6dbc4761 --- /dev/null +++ b/docs/cn/conf.py @@ -0,0 +1,289 @@ +# -*- coding: utf-8 -*- +# +# DPark documentation build configuration file, created by +# sphinx-quickstart on Mon Jan 25 17:28:47 2016. +# +# This file is execfile()d with the current directory set to its +# containing dir. +# +# Note that not all possible configuration values are present in this +# autogenerated file. +# +# All configuration values have a default; values that are commented out +# serve to show the default. + +import sys +import os + +# If extensions (or modules to document with autodoc) are in another directory, +# add these directories to sys.path here. If the directory is relative to the +# documentation root, use os.path.abspath to make it absolute, like shown here. +#sys.path.insert(0, os.path.abspath('.')) + +# -- General configuration ------------------------------------------------ + +# If your documentation needs a minimal Sphinx version, state it here. +#needs_sphinx = '1.0' + +# Add any Sphinx extension module names here, as strings. They can be +# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom +# ones. +extensions = [] + +# Add any paths that contain templates here, relative to this directory. +templates_path = ['_templates'] + +# The suffix(es) of source filenames. +# You can specify multiple suffix as a list of string: + +from recommonmark.parser import CommonMarkParser + +source_parsers = { + '.md': CommonMarkParser, +} + +source_suffix = ['.rst', '.md'] + +# The encoding of source files. +#source_encoding = 'utf-8-sig' + +# The master toctree document. +master_doc = 'index' + +# General information about the project. +project = u'DPark' +copyright = u'2016, Davies Liu, Zhongbo Tian' +author = u'Davies Liu, Zhongbo Tian' + +# The version info for the project you're documenting, acts as replacement for +# |version| and |release|, also used in various other places throughout the +# built documents. +# +# The short X.Y version. +version = u'0.2.6' +# The full version, including alpha/beta/rc tags. +release = u'0.2.6' + +# The language for content autogenerated by Sphinx. Refer to documentation +# for a list of supported languages. +# +# This is also used if you do content translation via gettext catalogs. +# Usually you set "language" from the command line for these cases. +language = 'cn' + +# There are two options for replacing |today|: either, you set today to some +# non-false value, then it is used: +#today = '' +# Else, today_fmt is used as the format for a strftime call. +#today_fmt = '%B %d, %Y' + +# List of patterns, relative to source directory, that match files and +# directories to ignore when looking for source files. +exclude_patterns = ['_build'] + +# The reST default role (used for this markup: `text`) to use for all +# documents. +#default_role = None + +# If true, '()' will be appended to :func: etc. cross-reference text. +#add_function_parentheses = True + +# If true, the current module name will be prepended to all description +# unit titles (such as .. function::). +#add_module_names = True + +# If true, sectionauthor and moduleauthor directives will be shown in the +# output. They are ignored by default. +#show_authors = False + +# The name of the Pygments (syntax highlighting) style to use. +pygments_style = 'sphinx' + +# A list of ignored prefixes for module index sorting. +#modindex_common_prefix = [] + +# If true, keep warnings as "system message" paragraphs in the built documents. +#keep_warnings = False + +# If true, `todo` and `todoList` produce output, else they produce nothing. +todo_include_todos = False + + +# -- Options for HTML output ---------------------------------------------- + +# The theme to use for HTML and HTML Help pages. See the documentation for +# a list of builtin themes. +html_theme = 'default' + +# Theme options are theme-specific and customize the look and feel of a theme +# further. For a list of options available for each theme, see the +# documentation. +#html_theme_options = {} + +# Add any paths that contain custom themes here, relative to this directory. +#html_theme_path = [] + +# The name for this set of Sphinx documents. If None, it defaults to +# " v documentation". +#html_title = None + +# A shorter title for the navigation bar. Default is the same as html_title. +#html_short_title = None + +# The name of an image file (relative to this directory) to place at the top +# of the sidebar. +#html_logo = None + +# The name of an image file (within the static path) to use as favicon of the +# docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 +# pixels large. +#html_favicon = None + +# Add any paths that contain custom static files (such as style sheets) here, +# relative to this directory. They are copied after the builtin static files, +# so a file named "default.css" will overwrite the builtin "default.css". +html_static_path = ['_static'] + +# Add any extra paths that contain custom files (such as robots.txt or +# .htaccess) here, relative to this directory. These files are copied +# directly to the root of the documentation. +#html_extra_path = [] + +# If not '', a 'Last updated on:' timestamp is inserted at every page bottom, +# using the given strftime format. +#html_last_updated_fmt = '%b %d, %Y' + +# If true, SmartyPants will be used to convert quotes and dashes to +# typographically correct entities. +#html_use_smartypants = True + +# Custom sidebar templates, maps document names to template names. +#html_sidebars = {} + +# Additional templates that should be rendered to pages, maps page names to +# template names. +#html_additional_pages = {} + +# If false, no module index is generated. +#html_domain_indices = True + +# If false, no index is generated. +#html_use_index = True + +# If true, the index is split into individual pages for each letter. +#html_split_index = False + +# If true, links to the reST sources are added to the pages. +#html_show_sourcelink = True + +# If true, "Created using Sphinx" is shown in the HTML footer. Default is True. +#html_show_sphinx = True + +# If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. +#html_show_copyright = True + +# If true, an OpenSearch description file will be output, and all pages will +# contain a tag referring to it. The value of this option must be the +# base URL from which the finished HTML is served. +#html_use_opensearch = '' + +# This is the file name suffix for HTML files (e.g. ".xhtml"). +#html_file_suffix = None + +# Language to be used for generating the HTML full-text search index. +# Sphinx supports the following languages: +# 'da', 'de', 'en', 'es', 'fi', 'fr', 'hu', 'it', 'ja' +# 'nl', 'no', 'pt', 'ro', 'ru', 'sv', 'tr' +#html_search_language = 'en' + +# A dictionary with options for the search language support, empty by default. +# Now only 'ja' uses this config value +#html_search_options = {'type': 'default'} + +# The name of a javascript file (relative to the configuration directory) that +# implements a search results scorer. If empty, the default will be used. +#html_search_scorer = 'scorer.js' + +# Output file base name for HTML help builder. +htmlhelp_basename = 'DParkdoc' + +# -- Options for LaTeX output --------------------------------------------- + +latex_elements = { +# The paper size ('letterpaper' or 'a4paper'). +#'papersize': 'letterpaper', + +# The font size ('10pt', '11pt' or '12pt'). +#'pointsize': '10pt', + +# Additional stuff for the LaTeX preamble. +#'preamble': '', + +# Latex figure (float) alignment +#'figure_align': 'htbp', +} + +# Grouping the document tree into LaTeX files. List of tuples +# (source start file, target name, title, +# author, documentclass [howto, manual, or own class]). +latex_documents = [ + (master_doc, 'DPark.tex', u'DPark Documentation', + u'Davies Liu, Zhongbo Tian', 'manual'), +] + +# The name of an image file (relative to this directory) to place at the top of +# the title page. +#latex_logo = None + +# For "manual" documents, if this is true, then toplevel headings are parts, +# not chapters. +#latex_use_parts = False + +# If true, show page references after internal links. +#latex_show_pagerefs = False + +# If true, show URL addresses after external links. +#latex_show_urls = False + +# Documents to append as an appendix to all manuals. +#latex_appendices = [] + +# If false, no module index is generated. +#latex_domain_indices = True + + +# -- Options for manual page output --------------------------------------- + +# One entry per manual page. List of tuples +# (source start file, name, description, authors, manual section). +man_pages = [ + (master_doc, 'dpark', u'DPark Documentation', + [author], 1) +] + +# If true, show URL addresses after external links. +#man_show_urls = False + + +# -- Options for Texinfo output ------------------------------------------- + +# Grouping the document tree into Texinfo files. List of tuples +# (source start file, target name, title, author, +# dir menu entry, description, category) +texinfo_documents = [ + (master_doc, 'DPark', u'DPark Documentation', + author, 'DPark', 'One line description of project.', + 'Miscellaneous'), +] + +# Documents to append as an appendix to all manuals. +#texinfo_appendices = [] + +# If false, no module index is generated. +#texinfo_domain_indices = True + +# How to display URL addresses: 'footnote', 'no', or 'inline'. +#texinfo_show_urls = 'footnote' + +# If true, do not generate a @detailmenu in the "Top" node's menu. +#texinfo_no_detailmenu = False diff --git a/docs/cn/faq.rst b/docs/cn/faq.rst new file mode 100644 index 00000000..a0e0f5e0 --- /dev/null +++ b/docs/cn/faq.rst @@ -0,0 +1,66 @@ +========= +Dpark FAQ +========= + +FAQ +=== + +1. 为什么我指定了'-m mesos'参数,但dpark似乎还是在单机模式运行? +--------------------------------------------------------------------- + +因为dpark的参数解析器看到第一个不能识别的参数就会放弃对后面参数的解析,所以‘-m mesos’这类的dpark参数请放在参数列表的最前面。另外dpark之所以不解析第一个不能识别的参数后面的参数,是为了支持‘drun ls -al’这类命令行的解析。 + +2. 为什么我的用dpark写的文件会损坏? +------------------------------------------ + +因为dpark并不保证每个task同时只有一个在运行,特别是当某个task执行的比这个 taskset 的其他task慢很多,dpark会尝试重新提交这个task,这样就会有相同的task在同时运行。此时当这个task有对共享存储的文件的写操作时,文件就很可能损坏。解决办法是在写文件时先写到一个文件名随机(或者用机器名+PID生成)的临时文件中,最后再将临时文件rename成最终文件 + +3. 为什么我在集群上的任务都会卡住或者失败? +-------------------------------------------------- + +因为你在计算集群上没有帐号,所以无法以自己的身份来提交计算任务,所以请用公共帐号mesos来提交,具体使用方式是sudo -u mesos 后面加上你的计算脚本命令行。 另一种解决方法是在 `helpdesk `_ 上申请 dpark_members 机群的权限。 + +4. saveAsTextFile会在目录中产生许许多多的小文件,能让文件变少点么? +------------------------------------------------------------------------ + +在saveAsTextFile之前用mergeSplit来减少输出文件数目,例如mergeSplit(3)会把原来每3个小文件合并为1个文件 + +5. parallelize 使用陷阱 +-------------------------- + +``parallelize(list_foo, num_split)`` 方式对 ``list_foo`` 进行分块时,会先将其传输到master节点,当 ``list_foo`` 非常大(say 100 MB)时,过长的网络传输时间可能会引发如下ERROR或WARNING: +``Lost Task`` , ``use too much time in slaveOffer`` , ``Connection dropped`` , ``Session has expired`` ,这时不管怎么调节 ``num_split`` 都是没有用的。一种可能的解决办法是将list_foo写入文本文件,最后以 ``textFile`` 读出进行分块。 + +6. 很多 “[WARNING] [dpark.taskset] Lost Task” +----------------------------------------------- + +原因可能因为你在计算集群上没有账号,解决方法参考问题 3。 + +7. 一些奇怪的模块依赖错误 +-------------------------------- + +原因可能是因为你的 DPark 脚本所在的文件夹或者你的 PYTHONPATH 路径上,有一些你自己编写的模块和标准库里的模块重名了 (e.g. email.py)。下面是个演示的例子,把它们放在同一文件夹下,然后执行 `python wc.py -m mesos` + +:: + + # email.py + import traceback + traceback.print_stack() + + + # wc.py + from dpark import DparkContext + import random + + ctx = DparkContext() + rdd = ctx.parallelize([random.randint(0, 10) for _ in range(100)], 5) + print rdd.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y).collect() + +8. 让人挠头的幽灵文件 +--------------------- + +你有一个通过 dpark saveAsTextFile 的输出,然后用一个其他的程序读取这个输出目录下的文件,结果程序奇怪的崩溃了。你发现是文件的输入格式不符合预期。你检查了你的 dpark 程序,但并没有什么收获。还尝试写了一个 dpark 程序想看看这些不符合预期的输入行,但是没有找到。正当你百思不得其解之际,你想起了 dpark 输出目录下的 tmp 文件。对的,就是他们。 + +dpark 同一个 task 会同时执行,但并不能保证这些临时输出的 tmp 文件会被清除掉,而 textFile 会主动忽略隐藏文件的。所以下次你可以主动清理一下这些 tmp 文件,或者在程序里做一些读入检查。不过你看到这一条的时候也许已经知道为什么了。 + + diff --git a/docs/guide_full.rst b/docs/cn/guide_full.rst similarity index 96% rename from docs/guide_full.rst rename to docs/cn/guide_full.rst index 6e23d8db..4ab85f18 100644 --- a/docs/guide_full.rst +++ b/docs/cn/guide_full.rst @@ -31,15 +31,13 @@ RDD(Resilient Distributed Datasets) 是 Dpark 的核心概念,是支持高容 - reduceByKey 会把数据的 key 拆分成 M 份进行计算,对每一份进行计算时,会把所有 key 放入内存,需要的内存量会比较大 - reduce 相当于只有一个 key,并且只有一份的 reduceByKey, 它最后是在当前进程进行结果合并的 -Stage / Job / Task +Stage / Taskset / Task ~~~~~~~~~~~~~~~~~~~~ - 由多个 RDD 组成的链式计算过程会根据依赖关系被划分为多个Stage - 每次 Shuffle 过程(由 combineByKey 产生)都会把计算过程拆分成前后两个 Stage -- 每个 Stage 会有多个 Job,对应一个或者多个类似的 RDD -- 一个 Job 有多个 Task,数量由 RDD 的 Split 决定 -- 同一个Stage里面的 Job 和 Task 是可以并行执行的 -- 每个 Task 对应一个进程,如果服务器资源充足,就可以完全并行,否则就只能部分并行 +- 每个 Stage 会 生成 一个 TaskSet, 其中的 task 并行执行,数量由 split 决定, 每个 task 处理一个分区的数据 +- 每个 Task 对应一个进程,如果服务器资源充足,task 们可以完全并行,否则就只能部分并行 共享变量 ~~~~~~~~~~~~~~~~~~~~ @@ -161,7 +159,7 @@ enumerate / enumeratePartition :: rdd = dpark.makeRDD(range(10), 5) - rdd.enumerate().collect() # [((0, 0), 0), ((0, 1), 1), ((1, 0), 2), ((1, 1), 3), ((2, 0), 4), ((2, 1), 5), ((3, 0), 6), ((3, 1), 7), ((4, 0), 8), ((4, 1), 9)] + rdd.enumerate().collect() # [(0, 0), (1, 1), (2, 2), (3, 3), (4, 4), (5, 5), (6, 6), (7, 7), (8, 8), (9, 9)] rdd.enumeratePartition().collect() # [(0, 0), (0, 1), (1, 2), (1, 3), (2, 4), (2, 5), (3, 6), (3, 7), (4, 8), (4, 9)] rdd.filter(lambda x: x > 3).enumeratePartition().collect() # [(2, 4), (2, 5), (3, 6), (3, 7), (4, 8), (4, 9)] @@ -368,7 +366,7 @@ bids 是一个 list,反复对 list 执行 in 操作,效率很低,转成 se ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ - 大部分 reduce 函数都支持设置 Task 数量[1]和 每个 Task 占用的内存,现在默认分别为 12 和 1000M -- 通常,一个脚本中的各个 Job 所需要的资源是不一样的,而 -M 参数会统一设置内存,所以建议复杂脚本不要使用 -M +- 通常,一个脚本中的各个 Stage 所需要的资源是不一样的,而 -M 参数会统一设置内存,所以建议复杂脚本不要使用 -M - Task 最大使用申请内存的 1.5 倍(将来会改成 1 倍),超过会失败,会在当前申请内存上乘 2 重试,最多重试 4 次,这个过程可以从 log 中看到 - 因为现在允许内存适当超标,所以也可能发生 Task 所在机器的内存不够而杀掉进程的情况 - 如果 log 中发现大量的内存报错,可以适当的增加 Task 和 Memroy diff --git a/docs/cn/index.rst b/docs/cn/index.rst new file mode 100644 index 00000000..e41a3f93 --- /dev/null +++ b/docs/cn/index.rst @@ -0,0 +1,27 @@ +.. DPark documentation master file, created by + sphinx-quickstart on Mon Jan 25 17:28:47 2016. + You can adapt this file completely to your liking, but it should at least + contain the root `toctree` directive. + +========== +Dpark 文档 +========== + +文档列表 +======== + +.. toctree:: + :maxdepth: 2 + :glob: + + * + + + +索引和表格 +========== + +* :ref:`genindex` +* :ref:`modindex` +* :ref:`search` + diff --git a/docs/en/Makefile b/docs/en/Makefile new file mode 100644 index 00000000..cdcf2e72 --- /dev/null +++ b/docs/en/Makefile @@ -0,0 +1,216 @@ +# Makefile for Sphinx documentation +# + +# You can set these variables from the command line. +SPHINXOPTS = +SPHINXBUILD = sphinx-build +PAPER = +BUILDDIR = _build + +# User-friendly check for sphinx-build +ifeq ($(shell which $(SPHINXBUILD) >/dev/null 2>&1; echo $$?), 1) +$(error The '$(SPHINXBUILD)' command was not found. Make sure you have Sphinx installed, then set the SPHINXBUILD environment variable to point to the full path of the '$(SPHINXBUILD)' executable. Alternatively you can add the directory with the executable to your PATH. If you don't have Sphinx installed, grab it from http://sphinx-doc.org/) +endif + +# Internal variables. +PAPEROPT_a4 = -D latex_paper_size=a4 +PAPEROPT_letter = -D latex_paper_size=letter +ALLSPHINXOPTS = -d $(BUILDDIR)/doctrees $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . +# the i18n builder cannot share the environment and doctrees with the others +I18NSPHINXOPTS = $(PAPEROPT_$(PAPER)) $(SPHINXOPTS) . + +.PHONY: help +help: + @echo "Please use \`make ' where is one of" + @echo " html to make standalone HTML files" + @echo " dirhtml to make HTML files named index.html in directories" + @echo " singlehtml to make a single large HTML file" + @echo " pickle to make pickle files" + @echo " json to make JSON files" + @echo " htmlhelp to make HTML files and a HTML help project" + @echo " qthelp to make HTML files and a qthelp project" + @echo " applehelp to make an Apple Help Book" + @echo " devhelp to make HTML files and a Devhelp project" + @echo " epub to make an epub" + @echo " latex to make LaTeX files, you can set PAPER=a4 or PAPER=letter" + @echo " latexpdf to make LaTeX files and run them through pdflatex" + @echo " latexpdfja to make LaTeX files and run them through platex/dvipdfmx" + @echo " text to make text files" + @echo " man to make manual pages" + @echo " texinfo to make Texinfo files" + @echo " info to make Texinfo files and run them through makeinfo" + @echo " gettext to make PO message catalogs" + @echo " changes to make an overview of all changed/added/deprecated items" + @echo " xml to make Docutils-native XML files" + @echo " pseudoxml to make pseudoxml-XML files for display purposes" + @echo " linkcheck to check all external links for integrity" + @echo " doctest to run all doctests embedded in the documentation (if enabled)" + @echo " coverage to run coverage check of the documentation (if enabled)" + +.PHONY: clean +clean: + rm -rf $(BUILDDIR)/* + +.PHONY: html +html: + $(SPHINXBUILD) -b html $(ALLSPHINXOPTS) $(BUILDDIR)/html + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/html." + +.PHONY: dirhtml +dirhtml: + $(SPHINXBUILD) -b dirhtml $(ALLSPHINXOPTS) $(BUILDDIR)/dirhtml + @echo + @echo "Build finished. The HTML pages are in $(BUILDDIR)/dirhtml." + +.PHONY: singlehtml +singlehtml: + $(SPHINXBUILD) -b singlehtml $(ALLSPHINXOPTS) $(BUILDDIR)/singlehtml + @echo + @echo "Build finished. The HTML page is in $(BUILDDIR)/singlehtml." + +.PHONY: pickle +pickle: + $(SPHINXBUILD) -b pickle $(ALLSPHINXOPTS) $(BUILDDIR)/pickle + @echo + @echo "Build finished; now you can process the pickle files." + +.PHONY: json +json: + $(SPHINXBUILD) -b json $(ALLSPHINXOPTS) $(BUILDDIR)/json + @echo + @echo "Build finished; now you can process the JSON files." + +.PHONY: htmlhelp +htmlhelp: + $(SPHINXBUILD) -b htmlhelp $(ALLSPHINXOPTS) $(BUILDDIR)/htmlhelp + @echo + @echo "Build finished; now you can run HTML Help Workshop with the" \ + ".hhp project file in $(BUILDDIR)/htmlhelp." + +.PHONY: qthelp +qthelp: + $(SPHINXBUILD) -b qthelp $(ALLSPHINXOPTS) $(BUILDDIR)/qthelp + @echo + @echo "Build finished; now you can run "qcollectiongenerator" with the" \ + ".qhcp project file in $(BUILDDIR)/qthelp, like this:" + @echo "# qcollectiongenerator $(BUILDDIR)/qthelp/DPark.qhcp" + @echo "To view the help file:" + @echo "# assistant -collectionFile $(BUILDDIR)/qthelp/DPark.qhc" + +.PHONY: applehelp +applehelp: + $(SPHINXBUILD) -b applehelp $(ALLSPHINXOPTS) $(BUILDDIR)/applehelp + @echo + @echo "Build finished. The help book is in $(BUILDDIR)/applehelp." + @echo "N.B. You won't be able to view it unless you put it in" \ + "~/Library/Documentation/Help or install it in your application" \ + "bundle." + +.PHONY: devhelp +devhelp: + $(SPHINXBUILD) -b devhelp $(ALLSPHINXOPTS) $(BUILDDIR)/devhelp + @echo + @echo "Build finished." + @echo "To view the help file:" + @echo "# mkdir -p $$HOME/.local/share/devhelp/DPark" + @echo "# ln -s $(BUILDDIR)/devhelp $$HOME/.local/share/devhelp/DPark" + @echo "# devhelp" + +.PHONY: epub +epub: + $(SPHINXBUILD) -b epub $(ALLSPHINXOPTS) $(BUILDDIR)/epub + @echo + @echo "Build finished. The epub file is in $(BUILDDIR)/epub." + +.PHONY: latex +latex: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo + @echo "Build finished; the LaTeX files are in $(BUILDDIR)/latex." + @echo "Run \`make' in that directory to run these through (pdf)latex" \ + "(use \`make latexpdf' here to do that automatically)." + +.PHONY: latexpdf +latexpdf: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through pdflatex..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +.PHONY: latexpdfja +latexpdfja: + $(SPHINXBUILD) -b latex $(ALLSPHINXOPTS) $(BUILDDIR)/latex + @echo "Running LaTeX files through platex and dvipdfmx..." + $(MAKE) -C $(BUILDDIR)/latex all-pdf-ja + @echo "pdflatex finished; the PDF files are in $(BUILDDIR)/latex." + +.PHONY: text +text: + $(SPHINXBUILD) -b text $(ALLSPHINXOPTS) $(BUILDDIR)/text + @echo + @echo "Build finished. The text files are in $(BUILDDIR)/text." + +.PHONY: man +man: + $(SPHINXBUILD) -b man $(ALLSPHINXOPTS) $(BUILDDIR)/man + @echo + @echo "Build finished. The manual pages are in $(BUILDDIR)/man." + +.PHONY: texinfo +texinfo: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo + @echo "Build finished. The Texinfo files are in $(BUILDDIR)/texinfo." + @echo "Run \`make' in that directory to run these through makeinfo" \ + "(use \`make info' here to do that automatically)." + +.PHONY: info +info: + $(SPHINXBUILD) -b texinfo $(ALLSPHINXOPTS) $(BUILDDIR)/texinfo + @echo "Running Texinfo files through makeinfo..." + make -C $(BUILDDIR)/texinfo info + @echo "makeinfo finished; the Info files are in $(BUILDDIR)/texinfo." + +.PHONY: gettext +gettext: + $(SPHINXBUILD) -b gettext $(I18NSPHINXOPTS) $(BUILDDIR)/locale + @echo + @echo "Build finished. The message catalogs are in $(BUILDDIR)/locale." + +.PHONY: changes +changes: + $(SPHINXBUILD) -b changes $(ALLSPHINXOPTS) $(BUILDDIR)/changes + @echo + @echo "The overview file is in $(BUILDDIR)/changes." + +.PHONY: linkcheck +linkcheck: + $(SPHINXBUILD) -b linkcheck $(ALLSPHINXOPTS) $(BUILDDIR)/linkcheck + @echo + @echo "Link check complete; look for any errors in the above output " \ + "or in $(BUILDDIR)/linkcheck/output.txt." + +.PHONY: doctest +doctest: + $(SPHINXBUILD) -b doctest $(ALLSPHINXOPTS) $(BUILDDIR)/doctest + @echo "Testing of doctests in the sources finished, look at the " \ + "results in $(BUILDDIR)/doctest/output.txt." + +.PHONY: coverage +coverage: + $(SPHINXBUILD) -b coverage $(ALLSPHINXOPTS) $(BUILDDIR)/coverage + @echo "Testing of coverage in the sources finished, look at the " \ + "results in $(BUILDDIR)/coverage/python.txt." + +.PHONY: xml +xml: + $(SPHINXBUILD) -b xml $(ALLSPHINXOPTS) $(BUILDDIR)/xml + @echo + @echo "Build finished. The XML files are in $(BUILDDIR)/xml." + +.PHONY: pseudoxml +pseudoxml: + $(SPHINXBUILD) -b pseudoxml $(ALLSPHINXOPTS) $(BUILDDIR)/pseudoxml + @echo + @echo "Build finished. The pseudo-XML files are in $(BUILDDIR)/pseudoxml." diff --git a/docs/en/conf.py b/docs/en/conf.py new file mode 100644 index 00000000..aba9a373 --- /dev/null +++ b/docs/en/conf.py @@ -0,0 +1,289 @@ +# -*- coding: utf-8 -*- +# +# DPark documentation build configuration file, created by +# sphinx-quickstart on Mon Jan 25 17:28:47 2016. +# +# This file is execfile()d with the current directory set to its +# containing dir. +# +# Note that not all possible configuration values are present in this +# autogenerated file. +# +# All configuration values have a default; values that are commented out +# serve to show the default. + +import sys +import os + +# If extensions (or modules to document with autodoc) are in another directory, +# add these directories to sys.path here. If the directory is relative to the +# documentation root, use os.path.abspath to make it absolute, like shown here. +#sys.path.insert(0, os.path.abspath('.')) + +# -- General configuration ------------------------------------------------ + +# If your documentation needs a minimal Sphinx version, state it here. +#needs_sphinx = '1.0' + +# Add any Sphinx extension module names here, as strings. They can be +# extensions coming with Sphinx (named 'sphinx.ext.*') or your custom +# ones. +extensions = [] + +# Add any paths that contain templates here, relative to this directory. +templates_path = ['_templates'] + +# The suffix(es) of source filenames. +# You can specify multiple suffix as a list of string: + +from recommonmark.parser import CommonMarkParser + +source_parsers = { + '.md': CommonMarkParser, +} + +source_suffix = ['.rst', '.md'] + +# The encoding of source files. +#source_encoding = 'utf-8-sig' + +# The master toctree document. +master_doc = 'index' + +# General information about the project. +project = u'DPark' +copyright = u'2016, Davies Liu, Zhongbo Tian' +author = u'Davies Liu, Zhongbo Tian' + +# The version info for the project you're documenting, acts as replacement for +# |version| and |release|, also used in various other places throughout the +# built documents. +# +# The short X.Y version. +version = u'0.2.6' +# The full version, including alpha/beta/rc tags. +release = u'0.2.6' + +# The language for content autogenerated by Sphinx. Refer to documentation +# for a list of supported languages. +# +# This is also used if you do content translation via gettext catalogs. +# Usually you set "language" from the command line for these cases. +language = 'en' + +# There are two options for replacing |today|: either, you set today to some +# non-false value, then it is used: +#today = '' +# Else, today_fmt is used as the format for a strftime call. +#today_fmt = '%B %d, %Y' + +# List of patterns, relative to source directory, that match files and +# directories to ignore when looking for source files. +exclude_patterns = ['_build'] + +# The reST default role (used for this markup: `text`) to use for all +# documents. +#default_role = None + +# If true, '()' will be appended to :func: etc. cross-reference text. +#add_function_parentheses = True + +# If true, the current module name will be prepended to all description +# unit titles (such as .. function::). +#add_module_names = True + +# If true, sectionauthor and moduleauthor directives will be shown in the +# output. They are ignored by default. +#show_authors = False + +# The name of the Pygments (syntax highlighting) style to use. +pygments_style = 'sphinx' + +# A list of ignored prefixes for module index sorting. +#modindex_common_prefix = [] + +# If true, keep warnings as "system message" paragraphs in the built documents. +#keep_warnings = False + +# If true, `todo` and `todoList` produce output, else they produce nothing. +todo_include_todos = False + + +# -- Options for HTML output ---------------------------------------------- + +# The theme to use for HTML and HTML Help pages. See the documentation for +# a list of builtin themes. +html_theme = 'default' + +# Theme options are theme-specific and customize the look and feel of a theme +# further. For a list of options available for each theme, see the +# documentation. +#html_theme_options = {} + +# Add any paths that contain custom themes here, relative to this directory. +#html_theme_path = [] + +# The name for this set of Sphinx documents. If None, it defaults to +# " v documentation". +#html_title = None + +# A shorter title for the navigation bar. Default is the same as html_title. +#html_short_title = None + +# The name of an image file (relative to this directory) to place at the top +# of the sidebar. +#html_logo = None + +# The name of an image file (within the static path) to use as favicon of the +# docs. This file should be a Windows icon file (.ico) being 16x16 or 32x32 +# pixels large. +#html_favicon = None + +# Add any paths that contain custom static files (such as style sheets) here, +# relative to this directory. They are copied after the builtin static files, +# so a file named "default.css" will overwrite the builtin "default.css". +html_static_path = ['_static'] + +# Add any extra paths that contain custom files (such as robots.txt or +# .htaccess) here, relative to this directory. These files are copied +# directly to the root of the documentation. +#html_extra_path = [] + +# If not '', a 'Last updated on:' timestamp is inserted at every page bottom, +# using the given strftime format. +#html_last_updated_fmt = '%b %d, %Y' + +# If true, SmartyPants will be used to convert quotes and dashes to +# typographically correct entities. +#html_use_smartypants = True + +# Custom sidebar templates, maps document names to template names. +#html_sidebars = {} + +# Additional templates that should be rendered to pages, maps page names to +# template names. +#html_additional_pages = {} + +# If false, no module index is generated. +#html_domain_indices = True + +# If false, no index is generated. +#html_use_index = True + +# If true, the index is split into individual pages for each letter. +#html_split_index = False + +# If true, links to the reST sources are added to the pages. +#html_show_sourcelink = True + +# If true, "Created using Sphinx" is shown in the HTML footer. Default is True. +#html_show_sphinx = True + +# If true, "(C) Copyright ..." is shown in the HTML footer. Default is True. +#html_show_copyright = True + +# If true, an OpenSearch description file will be output, and all pages will +# contain a tag referring to it. The value of this option must be the +# base URL from which the finished HTML is served. +#html_use_opensearch = '' + +# This is the file name suffix for HTML files (e.g. ".xhtml"). +#html_file_suffix = None + +# Language to be used for generating the HTML full-text search index. +# Sphinx supports the following languages: +# 'da', 'de', 'en', 'es', 'fi', 'fr', 'hu', 'it', 'ja' +# 'nl', 'no', 'pt', 'ro', 'ru', 'sv', 'tr' +#html_search_language = 'en' + +# A dictionary with options for the search language support, empty by default. +# Now only 'ja' uses this config value +#html_search_options = {'type': 'default'} + +# The name of a javascript file (relative to the configuration directory) that +# implements a search results scorer. If empty, the default will be used. +#html_search_scorer = 'scorer.js' + +# Output file base name for HTML help builder. +htmlhelp_basename = 'DParkdoc' + +# -- Options for LaTeX output --------------------------------------------- + +latex_elements = { +# The paper size ('letterpaper' or 'a4paper'). +#'papersize': 'letterpaper', + +# The font size ('10pt', '11pt' or '12pt'). +#'pointsize': '10pt', + +# Additional stuff for the LaTeX preamble. +#'preamble': '', + +# Latex figure (float) alignment +#'figure_align': 'htbp', +} + +# Grouping the document tree into LaTeX files. List of tuples +# (source start file, target name, title, +# author, documentclass [howto, manual, or own class]). +latex_documents = [ + (master_doc, 'DPark.tex', u'DPark Documentation', + u'Davies Liu, Zhongbo Tian', 'manual'), +] + +# The name of an image file (relative to this directory) to place at the top of +# the title page. +#latex_logo = None + +# For "manual" documents, if this is true, then toplevel headings are parts, +# not chapters. +#latex_use_parts = False + +# If true, show page references after internal links. +#latex_show_pagerefs = False + +# If true, show URL addresses after external links. +#latex_show_urls = False + +# Documents to append as an appendix to all manuals. +#latex_appendices = [] + +# If false, no module index is generated. +#latex_domain_indices = True + + +# -- Options for manual page output --------------------------------------- + +# One entry per manual page. List of tuples +# (source start file, name, description, authors, manual section). +man_pages = [ + (master_doc, 'dpark', u'DPark Documentation', + [author], 1) +] + +# If true, show URL addresses after external links. +#man_show_urls = False + + +# -- Options for Texinfo output ------------------------------------------- + +# Grouping the document tree into Texinfo files. List of tuples +# (source start file, target name, title, author, +# dir menu entry, description, category) +texinfo_documents = [ + (master_doc, 'DPark', u'DPark Documentation', + author, 'DPark', 'One line description of project.', + 'Miscellaneous'), +] + +# Documents to append as an appendix to all manuals. +#texinfo_appendices = [] + +# If false, no module index is generated. +#texinfo_domain_indices = True + +# How to display URL addresses: 'footnote', 'no', or 'inline'. +#texinfo_show_urls = 'footnote' + +# If true, do not generate a @detailmenu in the "Top" node's menu. +#texinfo_no_detailmenu = False diff --git a/docs/en/index.rst b/docs/en/index.rst new file mode 100644 index 00000000..990c7d90 --- /dev/null +++ b/docs/en/index.rst @@ -0,0 +1,27 @@ +.. DPark documentation master file, created by + sphinx-quickstart on Mon Jan 25 17:28:47 2016. + You can adapt this file completely to your liking, but it should at least + contain the root `toctree` directive. + +================================= +Welcome to DPark's documentation! +================================= + +Contents +======== + +.. toctree:: + :maxdepth: 2 + :glob: + + * + + + +Indices and tables +================== + +* :ref:`genindex` +* :ref:`modindex` +* :ref:`search` + diff --git a/docs/faq.rst b/docs/faq.rst deleted file mode 100644 index c1915f68..00000000 --- a/docs/faq.rst +++ /dev/null @@ -1,31 +0,0 @@ -========= -Dpark FAQ -========= - -FAQ -=== - -1. 为什么我指定了'-m mesos'参数,但dpark似乎还是在单机模式运行? ---------------------------------------------------------------------- - -因为dpark的参数解析器看到第一个不能识别的参数就会放弃对后面参数的解析,所以‘-m mesos’这类的dpark参数请放在参数列表的最前面。另外dpark之所以不解析第一个不能识别的参数后面的参数,是为了支持‘drun ls -al’这类命令行的解析。 - -2. 为什么我的用dpark写的文件会损坏? ------------------------------------------- - -因为dpark并不保证每个task同时只有一个在运行,特别是当某个task执行的比这个job的其他task慢很多,dpark会尝试重新提交这个task,这样就会有相同的task在同时运行。此时当这个task有对共享存储的文件的写操作时,文件就很可能损坏。解决办法是在写文件时先写到一个文件名随机(或者用机器名+PID生成)的临时文件中,最后再将临时文件rename成最终文件 - -3. 为什么我在集群上的任务都会卡住或者失败? --------------------------------------------------- - -因为你在计算集群上没有帐号,所以无法以自己的身份来提交计算任务,所以请用公共帐号mesos来提交。具体使用方式是sudo -u mesos 后面加上你的计算脚本命令行 - -4. saveAsTextFile会在目录中产生许许多多的小文件,能让文件变少点么? ------------------------------------------------------------------------- - -在saveAsTextFile之前用mergeSplit来减少输出文件数目,例如mergeSplit(3)会把原来每3个小文件合并为1个文件 - -5. parallelize 使用陷阱 --------------------------- -``parallelize(list_foo, num_split)`` 方式对 ``list_foo`` 进行分块时,会先将其传输到master节点,当 ``list_foo`` 非常大(say 100 MB)时,过长的网络传输时间可能会引发如下ERROR或WARNING: -``Lost Task`` , ``use too much time in slaveOffer`` , ``Connection dropped`` , ``Session has expired`` ,这时不管怎么调节 ``num_split`` 都是没有用的。一种可能的解决办法是将list_foo写入文本文件,最后以 ``textFile`` 读出进行分块。 \ No newline at end of file diff --git a/docs/guide_full.html b/docs/guide_full.html deleted file mode 100644 index eb91be0d..00000000 --- a/docs/guide_full.html +++ /dev/null @@ -1,681 +0,0 @@ - - - - - - -Dpark 使用指南 - - - -
-

Dpark 使用指南

- -
-

基本概念

-
-

RDD

-

RDD(Resilient Distributed Datasets) 是 Dpark 的核心概念,是支持高容错并行计算的数据集合

-
    -
  • 有两种方式可以产生 RDD,通过特定函数从存储设备(内存或硬盘)创建,或者由其他 RDD 生成
  • -
  • RDD 带有它所依赖的其他 RDD 的信息,以备计算失败的时候能够重新计算
  • -
  • RDD 是只读的,这样从拓扑中恢复这个 RDD 的工作就能简单很多
  • -
  • RDD 可以被重复使用
  • -
  • 一个 RDD 由多个 Split 组成,Split 是执行并行计算的基本单位
  • -
  • RDD 支持两类操作,窄依赖的 map 和 宽依赖的 reduce
  • -
-
-
-

窄依赖

-
    -
  • 窄依赖操作,会对每一行数据进行计算,类似 Python 中的 map 函数
  • -
  • 窄依赖操作是流式计算,需要的内存很少
  • -
  • 常用函数 map / flatMap / filter / mapValue
  • -
-
-
-

宽依赖

-
    -
  • 宽依赖操作需要所依赖的数据完全完成后,才能进行计算,所以对内存需求比较大
  • -
  • 常用函数 reduce / reduceByKey / uniq / groupBy / groupByKey / combineByKey
  • -
  • reduceByKey 会把数据的 key 拆分成 M 份进行计算,对每一份进行计算时,会把所有 key 放入内存,需要的内存量会比较大
  • -
  • reduce 相当于只有一个 key,并且只有一份的 reduceByKey, 它最后是在当前进程进行结果合并的
  • -
-
-
-

Stage / Job / Task

-
    -
  • 由多个 RDD 组成的链式计算过程会根据依赖关系被划分为多个Stage
  • -
  • 每次 Shuffle 过程(由 combineByKey 产生)都会把计算过程拆分成前后两个 Stage
  • -
  • 每个 Stage 会有多个 Job,对应一个或者多个类似的 RDD
  • -
  • 一个 Job 有多个 Task,数量由 RDD 的 Split 决定
  • -
  • 同一个Stage里面的 Job 和 Task 是可以并行执行的
  • -
  • 每个 Task 对应一个进程,如果服务器资源充足,就可以完全并行,否则就只能部分并行
  • -
-
-
-

共享变量

-
    -
  • Dpark 在运行时,会将 RDD 以及针对它的函数序列化后发送到执行节点去,反序列化并执行。函数所依赖的全局变量,模块和闭包对象等,也都会随着函数一块发送过去。
  • -
  • 每个 Task 都会发送一次,所以当依赖很大或者依赖中等但是 Task 很多,就会影响性能。这时候需要用到广播
  • -
-
-
-

广播

-
    -
  • 广播适用于比较大的数据集,但它不能超过单机的内存限制
  • -
  • 广播之后,使用时会在集群中各台机器之间交换数据,不会像序列化一样依赖于执行脚本的机器
  • -
  • 序列化后超过 100k 的对象都需要广播,我们称手工写 dpark.broadcast(xxx) 为显式广播
  • -
  • Dpark 可以自动的发现大对象并广播出去,代码中不需要手工写广播代码,我们称这种为隐式广播
  • -
  • Dpark 可能不能正确发现大对象,也有可能一个大对象同时被多个函数使用的情况,所以需要有选择的使用显式广播
  • -
-
-
-
-

命令行参数

-
-
-M xxxx
-
每个 task 申请的内存
-
--err 0.001
-
如果有脏数据,用这个可以忽略掉
-
-
-
-

常用函数

-
-

union

-
-rdd3 = rdd1.union(rdd2)
-rdd3 = dpark.union([rdd1, rdd2])
-
-
-
-

map / flatMap

-
-b = dpark.parallelize([1, 2])
-r1 = b.map(lambda x: (x, x)).collect()  # [(1, 1), (2, 2)]
-r2 = b.flatMap(lambda x: (x, x)).collect()  # [1, 1, 2, 2]
-
-
-
-

mapValue / flatMapValue

-
-b = dpark.parallelize([(1, 11), (2, 22)])
-r1 = b.mapValue(lambda x: (x, x)).collect()  # [(1, (11, 11)), (2, (22, 22))]
-r2 = b.flatMapValue(lambda x: (x, x)).collect()  # [(1, 11), (1, 11), (2, 22), (2, 22)]
-
-
-
-

filter

-
-b = dpark.parallelize([[1, 11], 0, [2, 22]])
-r = b.filter(lambda x: x).collect()  # [[1, 11], [2, 22]]
-
-
-
-

uniq

-
-b = dpark.parallelize([(1, 11), 0, (1, 11)])
-r = b.uniq().collect()  # [0, (1, 11)]
-
-
-
-

groupBy / groupByKey

-
-b = dpark.parallelize([(1, 11), (1, 12), (2, 22)])
-r1 = b.map(lambda x: (x[0], x)).groupByKey().collect()  # [(1, [(1, 12), (1, 11)]), (2, [(2, 22)])]
-r2 = b.groupBy(lambda x: x[0]).collect()  # [(1, [(1, 12), (1, 11)]), (2, [(2, 22)])]
-
-
-
-

reduce / reduceByKey

-
-b = dpark.parallelize([(1, 11), (1, 12), (2, 22)])
-r1 = b.reduceByKey(lambda x, y: x + y).collect()  # [(1, 23), (2, 22)]
-r2 = b.reduce(lambda x, y: (x[0] + y[0], x[1] + y[1]))  # (4, 45)
-
-
-
-

join / leftOuterJoin / rightOuterJoin / outerJoin / groupWith

-
-rdd1 = dpark.parallelize([(1, 11), (2, 12), (3, 22)])
-rdd2 = dpark.parallelize([(1, 33), (2, 44), (4, 55)])
-r1 = rdd1.join(rdd2).collect()  # [(1, (11, 33)), (2, (12, 44))]
-r2 = rdd1.leftOuterJoin(rdd2).collect()  # [(1, (11, 33)), (2, (12, 44)), (3, (22, None))]
-r3 = rdd1.rightOuterJoin(rdd2).collect()  # [(1, (11, 33)), (2, (12, 44)), (4, (None, 55))]
-r4 = rdd1.outerJoin(rdd2).collect()  # [(1, (11, 33)), (2, (12, 44)), (3, (22, None)), (4, (None, 55))]
-
-rdd3 = dpark.parallelize([(1, 100), (2, 101), (4, 201)])
-r5 = rdd1.groupWith(rdd2).collect()  # [(1, ([11], [33])), (2, ([12], [44])), (3, ([22], [])), (4, ([], [55]))]
-r6 = rdd1.groupWith([rdd2, rdd3]).collect()  # [(1, ([11], [33], [100])), (2, ([12], [44], [101])), (3, ([22], [], [])), (4, ([], [55], [201]))]
-
-
-
-

读相关

-
-textFile(self, path, ext='', followLink=True, maxdepth=0, cls=TextFileRDD, *ka, **kws)
-
-# 读单个文件,每个 Split 最大 16 M
-rdd = dpark.textFile('xxxx.csv', splitSize=16 << 20)
-
-# 读多个压缩文件(目前textFile支持 .bz2 和 .gz),每个文件分成 10 个 Split
-rdd = dpark.textFile(['xxxx.bz2', 'xxxxx.gz'], numSplits=10)
-
-# 递归读目录,扩展名为.csv,PS:隐藏文件会被忽略
-rdd = dpark.textFile('/xxxx/xxxx', ext='.csv')
-
-# 其他文件类型请参见 rdd.py,或使用 pydoc dpark.rdd
-
-
-
-

写相关

-
-# 写文件,扩展名.csv,gz 格式压缩
-rdd.saveAsTextFile(path, ext='.csv', compress=True)
-
-# 按 key 写入多个目录,扩展名.csv,path 下如已有文件则删除
-rdd = dpark.parallelize([('1', '1'), ('2', '2')])
-rdd.saveAsTextFileByKey(path, ext='.csv', overwrite=True) # path 下会生成 1 和 2 两个目录
-
-# 其他文件类型请参见 rdd.py,或使用 pydoc dpark.rdd
-
-
-
-
-

代码风格

-

我们先来看个例子

-
-data.map(
-    lambda line: line.strip().split(' ')
-).filter(
-    lambda line: len(line)>=3
-).map(
-    lambda line: (line[1],line[2])
-).map(
-    lambda line: (line[0].split(':'),line[1])
-).filter(
-    lambda line: len(line[0])>=2
-).map(
-    lambda line: (line[0][1],line[1]))
-
-

这种代码写起来方便,但是欠缺可读性。换个写法

-
-def split_row(r):
-    return r.strip().split(' ')
-
-def cal(r):
-    if len(r) < 3:
-        return
-
-    _, bus, date = r[:3]
-    t = bus.split(':')
-    if len(t) < 2:
-        return
-
-    return t[1], date
-
-data.map(split_row).map(cal).filter(lambda x: x)
-
-

上面的代码就会好很多

-
-
-

开发注意事项

-
    -
  • 先用小数据将代码调通,再执行大数据
  • -
  • 执行未调优的脚本要关注 log 中的警告和错误,随时准备停掉脚本
  • -
  • 务必以低并行度访问数据库,否则员外会找你喝茶
  • -
  • 执行 collect / collectAsMap 会将数据读入当前内存,建议先 saveAsTextFile 看看大小,确保不会过大
  • -
  • 了解自己的数据,才能有针对性的做优化
  • -
-
-
-

性能调优

-
-

优化非 Dpark 部分

-
    -
  • 先优化 map 依赖的函数,避免效率过低的操作,比如反复对大 list 执行 in 操作,反复的 re.compile 同一个表达式
  • -
  • 组织数据时,适当压缩大小,比如纯数字的字符串先转 int
  • -
-
-
-

使用广播的时机

-

一个简单的例子

-
-dpark = DparkContext()
-bid_data = dpark.parallelize(map(lambda x: (str(x), str(x)), range(10)))
-rdd = dpark.parallelize(map(lambda x: (str(x * 2), str(x)), range(100)))
-
-bids = bid_data.map(lambda r: r[1]).collect()
-r = rdd.filter(lambda r: r[1] in bids).collect()
-
-

bids 中的元素都是 string,如果条件允许而 bids 确实非常大,可以转成 int

-
-bids = dpark.parallelize(data).map(lambda r: int(r[1])).collect()
-
-

bids 是一个 list,反复对 list 执行 in 操作,效率很低,转成 set 或者 dict

-
-bids = set(bids)
-bids = dict(((u, 1) for u in bids))
-bids = bid_data.map(lambda r: int(r[1])).map(lambda x: (x, 1)).collectAsMap()
-
-

如果 bids 很大,就需要使用广播(Dpark 可能会在这里使用隐式广播)

-
-bids_b = dpark.broadcast(bids)
-r = rdd.filter(lambda r: int(r[1]) in bids_b.value).collect()
-
-

如果 bids 特别大,到了会影响网络 IO 的程度……

-
-bids = bid_data.map(lambda r: r[1]).map(lambda x: (x, 1))
-r = rdd.map(lambda r: (r[1], r)).join(bids).filter(lambda r: r[1][1]).map(lambda r: r[1][0]).collect()
-
-

视情况使用 leftOuterJoin 等,实战代码 /mfs/datasupport/xiliang_moria/agg_index_product_total_uv.py

-
-
-

尽快减小数据集

-
    -
  • 比如有两个独立操作 map 和 filter,先 filter 后 map 就可以减少一些不必要的计算
  • -
  • 同理,uniq 和 map 也可以如此处理
  • -
-
-
-

使用 groupBy / groupByKey 的注意事项

-
    -
  • 通常 key 小 value 大,所以不会大幅减少数据
  • -
  • 在 key 不均衡的情况下,会导致某个 task 过大而出错,极端情况脚本挂掉
  • -
  • 如果可能,优先使用 reduce 方式
  • -
-
-dpark = DparkContext()
-big_data = dpark.parallelize(range(10) + range(20) + range(30))
-
-r1 = big_data.groupBy(lambda x: x).mapValue(len).collect()
-r2 = big_data.map(lambda x: (x, 1)).groupByKey().mapValue(len).collect()
-
-

这两种做法都可能有上述隐患,更好的做法是

-
-r3 = big_data.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y).collect()
-
-

用 reduceByKey 来加快缩小数据。对合并后的 value 没整体需求的,都可以考虑用这种方式。

-
-
-

合理设置 Task 和 Memory

-
    -
  • 大部分 reduce 函数都支持设置 Task 数量和 每个 Task 占用的内存,现在默认分别为 12 和 1000M
  • -
  • 通常,一个脚本中的各个 Job 所需要的资源是不一样的,而 -M 参数会统一设置内存,所以建议复杂脚本不要使用 -M
  • -
  • Task 最大使用申请内存的 1.5 倍(将来会改成 1 倍),超过会失败,会在当前申请内存上乘 2 重试,最多重试 4 次,这个过程可以从 log 中看到
  • -
  • 因为现在允许内存适当超标,所以也可能发生 Task 所在机器的内存不够而杀掉进程的情况
  • -
  • 如果 log 中发现大量的内存报错,可以适当的增加 Task 和 Memroy
  • -
  • reduce 类的可以只增加 Task
  • -
  • groupBy 可能导致数据不平衡,需要兼顾 Task 和 Memory
  • -
  • 调整要逐步进行,重复进行“看警告,调参数”这个过程
  • -
-
-
-
-

一些实际的例子

-
-

延时计算陷阱

-

Dpark 是延时计算的,因此在使用结果的时候,要考虑是否已经计算过了

-
-dpark = DparkContext()
-rdd = dpark.parallelize(range(10))
-acc = dpark.accumulator(0)
-def sum(x):
-    acc.add(x)
-    return x
-
-rdd = rdd.map(sum)  # 如去掉赋值则属于无用代码
-print acc.value  # 0
-rdd.count()
-print acc.value  # 45
-
-
-
-

闭包陷阱

-

Python 本身的闭包可能会导致一些问题,开发的时候要注意一下

-
-from copy import copy
-dpark = DparkContext()
-# expect: [(0,0), (0,1), (1,1), (0,2), (1,2), (2,2)]
-
-rdd = dpark.union([dpark.makeRDD(range(i+1)).map(lambda x: (x,i)) for i in range(3)])
-print rdd.collect()  # but failed
-
-rdd = dpark.union([dpark.makeRDD(range(i+1)).map(lambda x: (x,copy(i))) for i in range(3)])
-print rdd.collect()  # still failed
-
-

这个问题是因为 Python 的变量绑定是语义范围,即闭包中的对象是由某个环境 + 变量名来决定的,而不是对象本身。一个解决办法是使用两层函数,另一个更简单的办法是使用函数的默认值,比如

-
-for i in range(10):
-   dpark.map((lambda i: lambda x: x + i)(i))  # 第一种方法,嵌套函数
-   dpark.map(lambda x,i=i: x + i)  # 第二种方法,默认值
-
-
-
-

合理使用 groupBy

-
    -
  • 也有必须使用 groupBy 的场合,比如使用 bid 计算 session
  • -
  • 还有需要利用 groupBy 来减少耗时操作的场合,比如现有 UA 库过慢,先对 UA 做 groupBy 以减少解析次数,或者对出现过多的 UA 预先进行解析,然后广播出去
  • -
  • 这种情况需要考虑数据不均衡的情况,大体思路都是拆分过大的 splits,但是仍然需要设置合适的 Memory
  • -
  • 具体例子可以看 /mfs/datasupport/xiliang_moria/fact_web_log2.py
  • -
-
-
-
- - diff --git a/docs/index.rst b/docs/index.rst deleted file mode 100644 index e06fc1b9..00000000 --- a/docs/index.rst +++ /dev/null @@ -1,12 +0,0 @@ -========== -Dpark 文档 -========== - -文档列表 -======== - -.. toctree:: - :maxdepth: 1 - :glob: - - * diff --git a/dpark/__init__.py b/dpark/__init__.py index d4cb1828..af2d0d1e 100644 --- a/dpark/__init__.py +++ b/dpark/__init__.py @@ -1,31 +1,5 @@ -from context import DparkContext, parser as optParser -from bagel import Bagel -from .decorator import jit, autojit +from __future__ import absolute_import +from .context import DparkContext, parser as optParser +from .bagel import Bagel -_ctx = DparkContext() - -parallelize = _ctx.parallelize - -makeRDD = _ctx.makeRDD - -textFile = _ctx.textFile - -partialTextFile = _ctx.partialTextFile - -csvFile = _ctx.csvFile - -binaryFile = _ctx.binaryFile - -tableFile = _ctx.tableFile - -beansdb = _ctx.beansdb - -union = _ctx.union - -zip = _ctx.zip - -start = _ctx.start - -stop = _ctx.stop - -clear = _ctx.clear +__all__ = ['DparkContext', 'optParser', 'Bagel'] diff --git a/dpark/accumulator.py b/dpark/accumulator.py index eeae79fc..c06f0a7f 100644 --- a/dpark/accumulator.py +++ b/dpark/accumulator.py @@ -1,8 +1,10 @@ +from __future__ import absolute_import from operator import add import copy from dpark.serialize import load_func, dump_func + class AccumulatorParam: def __init__(self, zero, addInPlace): self.zero = zero @@ -12,18 +14,19 @@ def __getstate__(self): return dump_func(self.addInPlace), self.zero def __setstate__(self, state): - add, self.zero = state - self.addInPlace = load_func(add) + add_, self.zero = state + self.addInPlace = load_func(add_) + numAcc = AccumulatorParam(0, add) -listAcc = AccumulatorParam([], lambda x,y:x.extend(y) or x) -mapAcc = AccumulatorParam({}, lambda x,y:x.update(y) or x) -setAcc = AccumulatorParam(set(), lambda x,y:x.update(y) or x) +listAcc = AccumulatorParam([], lambda x, y: x.extend(y) or x) +mapAcc = AccumulatorParam({}, lambda x, y: x.update(y) or x) +setAcc = AccumulatorParam(set(), lambda x, y: x.update(y) or x) class Accumulator: def __init__(self, initialValue=0, param=numAcc): - self.id = self.newId() + self.id = self.new_id() if param is None: param = numAcc self.param = param @@ -48,13 +51,15 @@ def __setstate__(self, s): self.register(self, False) nextId = 0 + @classmethod - def newId(cls): + def new_id(cls): cls.nextId += 1 return cls.nextId originals = {} localAccums = {} + @classmethod def register(cls, acc, original): if original: @@ -70,20 +75,18 @@ def clear(cls): @classmethod def values(cls): - v = dict((id, accum.value) for id,accum in cls.localAccums.items()) + v = dict((id_, accum.value) for id_, accum in cls.localAccums.items()) cls.clear() return v @classmethod def merge(cls, values): - for id, value in values.items(): - cls.originals[id].add(value) + for id_, value in values.items(): + cls.originals[id_].add(value) + ReadBytes = Accumulator() WriteBytes = Accumulator() -RemoteReadBytes = Accumulator() -LocalReadBytes = Accumulator() - CacheHits = Accumulator() CacheMisses = Accumulator() diff --git a/dpark/bagel.py b/dpark/bagel.py index f5a7d621..3ba99214 100644 --- a/dpark/bagel.py +++ b/dpark/bagel.py @@ -1,84 +1,104 @@ +from __future__ import absolute_import import sys -import os, os.path import time -import logging -import shutil import operator -logger = logging.getLogger(__name__) +from dpark.utils.log import get_logger + +logger = get_logger(__name__) + class Vertex: - def __init__(self, id, value, outEdges, active): - self.id = id + def __init__(self, id_, value, outEdges, active): + self.id = id_ self.value = value self.outEdges = outEdges self.active = active + def __repr__(self): return "" % (self.id, self.value, self.active) + class Edge: def __init__(self, target_id, value=0): self.target_id = target_id self.value = value + def __repr__(self): return '' % (self.target_id, self.value) + class Message: def __init__(self, target_id, value): self.target_id = target_id self.value = value + def __repr__(self): return "" % (self.target_id, self.value) class Combiner(object): def createCombiner(self, msg): raise NotImplementedError + def mergeValue(self, combiner, msg): raise NotImplementedError + def mergeCombiners(self, a, b): raise NotImplementedError + class Aggregator(object): def createAggregator(self, vert): raise NotImplementedError + def mergeAggregator(self, a, b): raise NotImplementedError + class BasicCombiner(Combiner): def __init__(self, op): self.op = op + def createCombiner(self, msg): return msg + def mergeValue(self, combiner, msg): return self.op(combiner, msg) + def mergeCombiners(self, a, b): return self.op(a, b) + DefaultValueCombiner = BasicCombiner(operator.add) + class DefaultListCombiner(Combiner): def createCombiner(self, msg): return [msg] + def mergeValue(self, combiner, msg): - return combiner+[msg] + return combiner + [msg] + def mergeCombiners(self, a, b): return a + b + class Bagel(object): @classmethod def run(cls, ctx, verts, msgs, compute, combiner=DefaultValueCombiner, aggregator=None, - max_superstep=sys.maxint, numSplits=None, snapshot_dir=None): + maxSuperstep=sys.maxsize, numSplits=None, checkpointDir=None): superstep = 0 - snapshot_dir = snapshot_dir or ctx.options.snapshot_dir + checkpointDir = checkpointDir or ctx.options.checkpoint_dir - while superstep < max_superstep: + while superstep < maxSuperstep: logger.info("Starting superstep %d", superstep) start = time.time() aggregated = cls.agg(verts, aggregator) if aggregator else None combinedMsgs = msgs.combineByKey(combiner, numSplits) grouped = verts.groupWith(combinedMsgs, numSplits=numSplits) verts, msgs, numMsgs, numActiveVerts = cls.comp(ctx, grouped, - lambda v, ms: compute(v, ms, aggregated, superstep), snapshot_dir) + lambda v, ms: compute(v, ms, aggregated, superstep), + checkpointDir) logger.info("superstep %d took %.1f s %d messages, %d active nodes", - superstep, time.time()-start, numMsgs, numActiveVerts) + superstep, time.time() - start, numMsgs, numActiveVerts) superstep += 1 if numMsgs == 0 and numActiveVerts == 0: @@ -87,14 +107,16 @@ def run(cls, ctx, verts, msgs, compute, @classmethod def agg(cls, verts, aggregator): - r = verts.map(lambda (id, vert): aggregator.createAggregator(vert)) + r = verts.map(lambda id_vert: aggregator.createAggregator(id_vert[1])) return r.reduce(aggregator.mergeAggregators) @classmethod - def comp(cls, ctx, grouped, compute, snapshot_dir=None): + def comp(cls, ctx, grouped, compute, checkpointDir=None): numMsgs = ctx.accumulator(0) numActiveVerts = ctx.accumulator(0) - def proc((vs, cs)): + + def proc(vs_cs): + (vs, cs) = vs_cs if not vs: return [] newVert, newMsgs = compute(vs[0], cs) @@ -102,19 +124,21 @@ def proc((vs, cs)): if newVert.active: numActiveVerts.add(1) return [(newVert, newMsgs)] + processed = grouped.flatMapValue(proc) - verts = processed.mapValue(lambda (vert, msgs): vert) - msgs = processed.flatMap(lambda (id, (vert, msgs)): msgs) - if snapshot_dir: - verts = verts.snapshot(snapshot_dir) - #else: + verts = processed.mapValue(lambda vert_msgs: vert_msgs[0]) + msgs = processed.flatMap(lambda id_vert_msgs: id_vert_msgs[1][1]) + if checkpointDir: + verts = verts.checkpoint(checkpointDir) + # else: # processed = processed.cache() # force evaluation of processed RDD for accurate performance measurements - n = verts.count() + _ = verts.count() return verts, msgs, numMsgs.value, numActiveVerts.value @classmethod def addAggregatorArg(cls, compute): def _(vert, messages, aggregator, superstep): return compute(vert, messages) + return _ diff --git a/dpark/broadcast.py b/dpark/broadcast.py index 22283ff3..1100b2dd 100644 --- a/dpark/broadcast.py +++ b/dpark/broadcast.py @@ -1,325 +1,643 @@ +from __future__ import absolute_import import os import zmq +import uuid as uuid_pkg import time -import uuid -import zlib +import binascii import random import socket import struct -import cPickle -import logging import marshal +import mmap +from multiprocessing import Manager, Condition +from mmap import ACCESS_WRITE, ACCESS_READ -from dpark.util import compress, decompress, spawn +from dpark.utils.log import get_logger +from dpark.utils import compress, decompress, spawn from dpark.cache import Cache from dpark.serialize import marshalable from dpark.env import env +import six +from six.moves import range, map, cPickle -logger = logging.getLogger("broadcast") +try: + from itertools import izip +except ImportError: + izip = zip -MARSHAL_TYPE, PICKLE_TYPE = range(2) +logger = get_logger(__name__) + +MARSHAL_TYPE, PICKLE_TYPE = list(range(2)) BLOCK_SHIFT = 20 BLOCK_SIZE = 1 << BLOCK_SHIFT +GUIDE_ADDR = 'NewBroadcastGuideAddr' +DOWNLOAD_ADDR = 'NewDownloadAddr' +BATCHED_BLOCKS = 3 +GUIDE_STOP, GUIDE_GET_SOURCES, GUIDE_SET_SOURCES, GUIDE_REPORT_BAD = list(range(4)) +SERVER_STOP, SERVER_FETCH, SERVER_FETCH_FAIL, SERVER_FETCH_OK, \ +DATA_GET, DATA_GET_OK, DATA_GET_FAIL, DATA_DOWNLOADING, SERVER_CLEAR_ITEM = list(range(9)) -class BroadcastManager: - header_fmt = '>BI' - header_len = struct.calcsize(header_fmt) - - def start(self, is_master): - raise NotImplementedError - def shutdown(self): - raise NotImplementedError - - def register(self, uuid, value): - raise NotImplementedError - - def clear(self, uuid): - raise NotImplementedError - - def fetch(self, uuid, block_num): - raise NotImplementedError - - def to_blocks(self, uuid, obj): - try: - if marshalable(obj): - buf = marshal.dumps((uuid, obj)) - type = MARSHAL_TYPE - else: - buf = cPickle.dumps((uuid, obj), -1) - type = PICKLE_TYPE +class GuideManager(object): + def __init__(self): + self._started = False + self.guides = {} + self.host = socket.gethostname() + self.guide_thread = None + self.guide_addr = None + self.register_addr = {} + self.ctx = zmq.Context() - except Exception: - buf = cPickle.dumps((uuid, obj), -1) - type = PICKLE_TYPE + def start(self): + if self._started: + return - checksum = zlib.crc32(buf) & 0xFFFF - stream = struct.pack(self.header_fmt, type, checksum) + buf - blockNum = (len(stream) + (BLOCK_SIZE - 1)) >> BLOCK_SHIFT - blocks = [compress(stream[i*BLOCK_SIZE:(i+1)*BLOCK_SIZE]) for i in range(blockNum)] - return blocks + self._started = True + self.guide_thread = self.start_guide() + env.register(GUIDE_ADDR, self.guide_addr) - def from_blocks(self, uuid, blocks): - stream = ''.join(map(decompress, blocks)) - type, checksum = struct.unpack(self.header_fmt, stream[:self.header_len]) - buf = stream[self.header_len:] - _checksum = zlib.crc32(buf) & 0xFFFF - if _checksum != checksum: - raise RuntimeError('Wrong blocks: checksum: %s, expected: %s' % ( - _checksum, checksum)) + def start_guide(self): + sock = self.ctx.socket(zmq.REP) + port = sock.bind_to_random_port('tcp://0.0.0.0') + self.guide_addr = 'tcp://%s:%d' % (self.host, port) - if type == MARSHAL_TYPE: - _uuid, value = marshal.loads(buf) - elif type == PICKLE_TYPE: - _uuid, value = cPickle.loads(buf) - else: - raise RuntimeError('Unknown serialization type: %s' % type) + def run(): + logger.debug("guide start at %s", self.guide_addr) - if uuid != _uuid: - raise RuntimeError('Wrong blocks: uuid: %s, expected: %s' % (_uuid, uuid)) + while self._started: + if not sock.poll(1000, zmq.POLLIN): + continue - return value + type_, msg = sock.recv_pyobj() + if type_ == GUIDE_STOP: + sock.send_pyobj(0) + break + elif type_ == GUIDE_GET_SOURCES: + uuid = msg + sources = None + if uuid in self.guides: + sources = self.guides[uuid] + else: + logger.warning('uuid %s NOT REGISTERED in guide server', uuid) + sock.send_pyobj(sources) + elif type_ == GUIDE_SET_SOURCES: + uuid, addr, bitmap = msg + if any(bitmap): + sources = None + if uuid in self.guides: + sources = self.guides[uuid] + if sources: + sources[addr] = bitmap + else: + self.guides[uuid] = {addr: bitmap} + self.register_addr[uuid] = addr + sock.send_pyobj(None) + elif type_ == GUIDE_REPORT_BAD: + uuid, addr = msg + sources = self.guides[uuid] + if addr in sources: + if addr != self.register_addr[uuid]: + del sources[addr] + else: + logger.warning('The addr %s to delete is the register Quit!!!', addr) + sock.send_pyobj(None) + else: + logger.error('Unknown guide message: %s %s', type_, msg) + sock.send_pyobj(None) -GUIDE_STOP, GUIDE_INFO, GUIDE_SOURCES, GUIDE_REPORT_BAD = range(4) -SERVER_STOP, SERVER_FETCH, SERVER_FETCH_FAIL, SERVER_FETCH_OK = range(4) + return spawn(run) -class P2PBroadcastManager(BroadcastManager): + def shutdown(self): + if not self._started: + return + + self._started = False + if self.guide_thread and self.guide_addr. \ + startswith('tcp://%s:' % socket.gethostname()): + self.guide_thread.join(timeout=1) + if self.guide_thread.is_alive(): + logger.warning("guide_thread not stopped.") + self.guide_addr = None + + +def check_memory(location): + try: + import psutil + pid = os.getpid() + p = psutil.Process(pid) + rss = p.memory_info().rss >> 20 + logger.info('memory rss %d MB in host %s at ', + rss, socket.gethostname(), location) + except ImportError: + logger.warning('import psutil failed') + + +class DownloadManager(object): def __init__(self): - self.published = {} - self.cache = Cache() - self.host = socket.gethostname() + self._started = False self.server_thread = None - random.seed(os.getpid() + int(time.time()*1000)%1000) - - def start(self, is_master): - if is_master: - self.guides = {} - self.guide_addr, self.guide_thread = self.start_guide() - env.register('BroadcastGuideAddr', self.guide_addr) - else: - self.guide_addr = env.get('BroadcastGuideAddr') + self.download_threads = {} + self.uuid_state_dict = None + self.uuid_map_dict = None + self.guide_addr = None + self.server_addr = None + self.host = None + self.ctx = None + self.random_inst = None + self.master_broadcast_blocks = {} + + def start(self): + if self._started: + return + + self.manager = manager = Manager() + self.shared_uuid_fn_dict = manager.dict() + self.shared_uuid_map_dict = manager.dict() + self.shared_master_blocks = manager.dict() + self.download_cond = Condition() + + self._started = True + self.ctx = zmq.Context() + self.host = socket.gethostname() + if GUIDE_ADDR not in env.environ: + start_guide_manager() - logger.debug("broadcast started: %s", self.guide_addr) + self.guide_addr = env.get(GUIDE_ADDR) + self.random_inst = random.SystemRandom() + self.server_addr, self.server_thread = self.start_server() + self.uuid_state_dict = {} + self.uuid_map_dict = {} + self.master_broadcast_blocks = {} + env.register(DOWNLOAD_ADDR, self.server_addr) - def shutdown(self): - sock = env.ctx.socket(zmq.REQ) + def start_server(self): + sock = self.ctx.socket(zmq.REP) sock.setsockopt(zmq.LINGER, 0) - sock.connect(self.guide_addr) - sock.send_pyobj((GUIDE_STOP, None)) - sock.recv_pyobj() - sock.close() - - def register(self, uuid, value): - if uuid in self.published: - raise RuntimeError('broadcast %s has already registered' % uuid) + port = sock.bind_to_random_port("tcp://0.0.0.0") + server_addr = 'tcp://%s:%d' % (self.host, port) + guide_sock = self.ctx.socket(zmq.REQ) + guide_sock.setsockopt(zmq.LINGER, 0) + guide_sock.connect(self.guide_addr) - if not self.server_thread: - self.server_addr, self.server_thread = self.start_server() + def run(): + logger.debug("server started at %s", server_addr) - blocks = self.to_blocks(uuid, value) - self.published[uuid] = blocks - self.guides[uuid] = {self.server_addr: [1] * len(blocks)} - self.cache.put(uuid, value) - return len(blocks) + while self._started: + if not sock.poll(1000, zmq.POLLIN): + continue + type_, msg = sock.recv_pyobj() + logger.debug('server recv: %s %s', type_, msg) + if type_ == SERVER_STOP: + sock.send_pyobj(None) + break + elif type_ == SERVER_FETCH: + uuid, indices, client_addr = msg + if uuid in self.master_broadcast_blocks: + block_num = len(self.master_broadcast_blocks[uuid]) + bls = [] + for index in indices: + if index >= block_num: + logger.warning('input index too big %s for ' + 'len of blocks %d from host %s', + str(indices), block_num, client_addr) + sock.send_pyobj((SERVER_FETCH_FAIL, None)) + else: + bls.append(self.master_broadcast_blocks[uuid][index]) + sock.send_pyobj((SERVER_FETCH_OK, (indices, bls))) + elif uuid in self.uuid_state_dict: + fd = os.open(self.uuid_state_dict[uuid][0], os.O_RDONLY) + mmfp = mmap.mmap(fd, 0, access=ACCESS_READ) + os.close(fd) + bitmap = self.uuid_map_dict[uuid] + block_num = len(bitmap) + bls = [] + for index in indices: + if index >= block_num: + logger.warning('input index too big %s for ' + 'len of blocks %d from host %s', + str(indices), block_num, client_addr) + sock.send_pyobj((SERVER_FETCH_FAIL, None)) + else: + mmfp.seek(bitmap[index][0]) + block = mmfp.read(bitmap[index][1]) + bls.append(block) + mmfp.close() + sock.send_pyobj((SERVER_FETCH_OK, (indices, bls))) + else: + logger.warning('server fetch failed for uuid %s ' + 'not exists in server %s from host %s', + uuid, socket.gethostname(), client_addr) + sock.send_pyobj((SERVER_FETCH_FAIL, None)) + elif type_ == DATA_GET: + uuid, compressed_size = msg + if uuid not in self.uuid_state_dict or not self.uuid_state_dict[uuid][1]: + if uuid not in self.download_threads: + sources = self._get_sources(uuid, guide_sock) + if not sources: + logger.warning('get sources from guide server failed in host %s', + socket.gethostname()) + sock.send_pyobj(DATA_GET_FAIL) + continue + self.download_threads[uuid] = spawn(self._download_blocks, + *[sources, uuid, compressed_size]) + sock.send_pyobj(DATA_DOWNLOADING) + else: + sock.send_pyobj(DATA_DOWNLOADING) + else: + sock.send_pyobj(DATA_GET_OK) + elif type_ == SERVER_CLEAR_ITEM: + uuid = msg + self.clear(uuid) + sock.send_pyobj(None) + else: + logger.error('Unknown server message: %s %s', type_, msg) + sock.send_pyobj(None) - def fetch(self, uuid, block_num): - if not self.server_thread: - self.server_addr, self.server_thread = self.start_server() + sock.close() + logger.debug("stop Broadcast server %s", server_addr) + for uuid in list(self.uuid_state_dict.keys()): + self.clear(uuid) - value = self.cache.get(uuid) - if value is not None: - return value + return server_addr, spawn(run) - blocks = self.fetch_blocks(uuid, block_num) - value = self.from_blocks(uuid, blocks) - return value + def get_blocks(self, uuid): + if uuid in self.master_broadcast_blocks: + return self.master_broadcast_blocks[uuid] + if uuid in self.shared_master_blocks: + return self.shared_master_blocks[uuid] - def clear(self, uuid): - self.cache.put(uuid, None) - del self.published[uuid] + def register_blocks(self, uuid, blocks): + if uuid in self.master_broadcast_blocks: + logger.warning('the block uuid %s exists in dict', uuid) + return + self.master_broadcast_blocks[uuid] = blocks + self.shared_master_blocks[uuid] = blocks - def fetch_blocks(self, uuid, block_num): - guide_sock = env.ctx.socket(zmq.REQ) - guide_sock.connect(self.guide_addr) - logger.debug("connect to guide %s", self.guide_addr) + def _get_sources(self, uuid, source_sock): + try: + source_sock.send_pyobj((GUIDE_GET_SOURCES, + uuid)) + sources = source_sock.recv_pyobj() + except: + logger.warning('GET sources failed for addr %s with ZMQ ERR', + self.server_addr) + sources = {} + return sources + + def _update_sources(self, uuid, bitmap, source_sock): + try: + source_sock.send_pyobj((GUIDE_SET_SOURCES, + (uuid, self.server_addr, bitmap))) + source_sock.recv_pyobj() + except: + pass + + def _download_blocks(self, sources, uuid, compressed_size): + block_num = 0 + bitmap = [0] + write_mmap_handler = None + download_guide_sock = self.ctx.socket(zmq.REQ) + download_guide_sock.setsockopt(zmq.LINGER, 0) + download_guide_sock.connect(self.guide_addr) - blocks = [None] * block_num - bitmap = [0] * block_num - self.published[uuid] = blocks def _report_bad(addr): - guide_sock.send_pyobj((GUIDE_REPORT_BAD, (uuid, addr))) - guide_sock.recv_pyobj() + logger.debug('fetch blocks failed from server %s', addr) + download_guide_sock.send_pyobj((GUIDE_REPORT_BAD, (uuid, addr))) + download_guide_sock.recv_pyobj() - def _fetch(addr, indices): - sock = env.ctx.socket(zmq.REQ) + def _fetch(addr, indices, bit_map): + sock = self.ctx.socket(zmq.REQ) try: sock.setsockopt(zmq.LINGER, 0) sock.connect(addr) - for i in indices: - sock.send_pyobj((SERVER_FETCH, (uuid, i))) - avail = sock.poll(5 * 1000, zmq.POLLIN) - if not avail: - logger.debug("%s recv broadcast %d from %s timeout", - self.server_addr, i, addr) - _report_bad(addr) - return - result, msg = sock.recv_pyobj() - - if result == SERVER_FETCH_FAIL: + sock.send_pyobj((SERVER_FETCH, (uuid, indices, self.server_addr))) + avail = sock.poll(1 * 1000, zmq.POLLIN) + check_sock = None + if not avail: + try: + check_sock = socket.socket() + addr_list = addr[len('tcp://'):].split(':') + addr_list[1] = int(addr_list[1]) + check_sock.connect(tuple(addr_list)) + except Exception as e: + logger.warning('connect the addr %s failed with exception %s', + addr, e) _report_bad(addr) - return - if result == SERVER_FETCH_OK: - id, block = msg - if i == id and block is not None: - blocks[id] = block - bitmap[id] = 1 else: - raise RuntimeError('Unknown server response: %s %s' % (result, msg)) - + logger.debug("%s recv broadcast %s from %s timeout", + self.server_addr, str(indices), addr) + finally: + if check_sock: + check_sock.close() + return + result, msg = sock.recv_pyobj() + if result == SERVER_FETCH_FAIL: + _report_bad(addr) + return + if result == SERVER_FETCH_OK: + indices, blocks = msg + for rank, index in enumerate(indices): + if blocks[rank] is not None: + write_mmap_handler.seek(bit_map[index][0]) + write_mmap_handler.write(blocks[rank]) + bitmap[index] = bit_map[index] + else: + raise RuntimeError('Unknown server response: %s %s' % (result, msg)) finally: sock.close() + final_path = env.workdir.alloc_tmp_file("broadcast") + self.uuid_state_dict[uuid] = (final_path, False) + fp = open(final_path, 'wb') + fp.truncate(compressed_size) + fp.close() + fd = os.open(final_path, os.O_RDWR) + write_mmap_handler = mmap.mmap(fd, 0, + access=ACCESS_WRITE) + os.close(fd) while not all(bitmap): - guide_sock.send_pyobj((GUIDE_SOURCES, (uuid, self.server_addr, bitmap))) - sources = guide_sock.recv_pyobj() - logger.debug("received SourceInfo from master: %s", sources.keys()) - local = [] remote = [] - for addr, _bitmap in sources.iteritems(): - if addr.startswith('tcp://%s:' % self.host): - local.append((addr, _bitmap)) - else: - remote.append((addr, _bitmap)) + for _addr, _bitmap in six.iteritems(sources): + if block_num == 0: + block_num = len(_bitmap) + bitmap = [0] * block_num + self.uuid_map_dict[uuid] = bitmap + if not _addr.startswith('tcp://%s:' % self.host): + remote.append((_addr, _bitmap)) + self.random_inst.shuffle(remote) + for _addr, _bitmap in remote: + _indices = [i for i in range(block_num) if not bitmap[i] and _bitmap[i]] + if _indices: + self.random_inst.shuffle(_indices) + _fetch(_addr, _indices[:BATCHED_BLOCKS], _bitmap) + self._update_sources(uuid, bitmap, download_guide_sock) + sources = self._get_sources(uuid, download_guide_sock) + write_mmap_handler.flush() + write_mmap_handler.close() + self.shared_uuid_map_dict[uuid] = bitmap + self.shared_uuid_fn_dict[uuid] = self.uuid_state_dict[uuid][0] + self.uuid_state_dict[uuid] = self.uuid_state_dict[uuid][0], True + download_guide_sock.close() + with self.download_cond: + self.download_cond.notify_all() - for addr, _bitmap in local: - indices = [i for i in xrange(block_num) if not bitmap[i] and _bitmap[i]] - if indices: - _fetch(addr, indices) + def clear(self, uuid): + if uuid in self.master_broadcast_blocks: + del self.master_broadcast_blocks[uuid] + del self.shared_master_blocks[uuid] + if uuid in self.uuid_state_dict: + del self.uuid_state_dict[uuid] + if uuid in self.shared_uuid_fn_dict: + del self.shared_uuid_fn_dict[uuid] + del self.shared_uuid_map_dict[uuid] - random.shuffle(remote) - for addr, _bitmap in remote: - indices = [i for i in xrange(block_num) if not bitmap[i] and _bitmap[i]] - if indices: - _fetch(addr, [random.choice(indices)]) + def shutdown(self): + if not self._started: + return - guide_sock.close() - return blocks + self._started = False + if self.server_thread and self.server_addr. \ + startswith('tcp://%s:' % socket.gethostname()): + for _, th in six.iteritems(self.download_threads): + th.join(timeout=0.1) # only in executor, not needed + self.server_thread.join(timeout=1) + if self.server_thread.is_alive(): + logger.warning("Download mananger server_thread not stopped.") - def start_guide(self): - sock = env.ctx.socket(zmq.REP) - port = sock.bind_to_random_port("tcp://0.0.0.0") - guide_addr = "tcp://%s:%d" % (self.host, port) + self.manager.shutdown() # shutdown will try join and terminate server process - def run(): - logger.debug("guide start at %s", guide_addr) - while True: - type, msg = sock.recv_pyobj() - if type == GUIDE_STOP: - sock.send_pyobj(0) - break - elif type == GUIDE_SOURCES: - uuid, addr, bitmap = msg - sources = self.guides[uuid] - sock.send_pyobj(sources) - if any(bitmap): - sources[addr] = bitmap - elif type == GUIDE_REPORT_BAD: - uuid, addr = msg - sock.send_pyobj(0) - sources = self.guides[uuid] - if addr in sources: - del sources[addr] - else: - logger.error('Unknown guide message: %s %s', type, msg) +def accumulate_list(l): + acc = 0 + acc_l = [] + for item in l: + acc_l.append(acc) + acc += item + acc_l.append(acc) + return acc_l - sock.close() - logger.debug("Sending stop notification to all servers ...") - for uuid, sources in self.guides.iteritems(): - for addr in sources: - self.stop_server(addr) - return guide_addr, spawn(run) +class BroadcastManager(object): + header_fmt = '>BI' + header_len = struct.calcsize(header_fmt) - def start_server(self): - sock = env.ctx.socket(zmq.REP) - sock.setsockopt(zmq.LINGER, 0) - port = sock.bind_to_random_port("tcp://0.0.0.0") - server_addr = 'tcp://%s:%d' % (self.host,port) + def __init__(self): + self._started = False + self.guide_addr = None + self.download_addr = None + self.cache = None + self.shared_uuid_fn_dict = None + self.shared_uuid_map_dict = None + self.download_cond = None + self.ctx = None + + def start(self): + if self._started: + return + + self._started = True + start_download_manager() + self.guide_addr = env.get(GUIDE_ADDR) + self.download_addr = env.get(DOWNLOAD_ADDR) + self.cache = Cache() + self.ctx = zmq.Context() + self.shared_uuid_fn_dict = _download_manager.shared_uuid_fn_dict + self.shared_uuid_map_dict = _download_manager.shared_uuid_map_dict + self.download_cond = _download_manager.download_cond - def run(): - logger.debug("server started at %s", server_addr) + def register(self, uuid, value): + self.start() - while True: - type, msg = sock.recv_pyobj() - logger.debug('server recv: %s %s', type, msg) - if type == SERVER_STOP: - sock.send_pyobj(None) - break - elif type == SERVER_FETCH: - uuid, id = msg - if uuid not in self.published: - sock.send_pyobj((SERVER_FETCH_FAIL, None)) - else: - blocks = self.published[uuid] - if id >= len(blocks): - sock.send_pyobj((SERVER_FETCH_FAIL, None)) - else: - sock.send_pyobj((SERVER_FETCH_OK, (id, blocks[id]))) + if uuid in self.shared_uuid_fn_dict: + raise RuntimeError('broadcast %s has already registered' % uuid) + blocks, size, block_map = self.to_blocks(uuid, value) + _download_manager.register_blocks(uuid, blocks) + self._update_sources(uuid, block_map) + self.cache.put(uuid, value) + return size + + def _update_sources(self, uuid, bitmap): + guide_sock = self.ctx.socket(zmq.REQ) + try: + guide_sock.setsockopt(zmq.LINGER, 0) + guide_sock.connect(self.guide_addr) + guide_sock.send_pyobj((GUIDE_SET_SOURCES, + (uuid, self.download_addr, bitmap))) + guide_sock.recv_pyobj() + finally: + guide_sock.close() + + def clear(self, uuid): + assert self._started + self.cache.put(uuid, None) + sock = self.ctx.socket(zmq.REQ) + sock.connect(self.download_addr) + sock.send_pyobj((SERVER_CLEAR_ITEM, uuid)) + sock.recv_pyobj() + sock.close() + + def fetch(self, uuid, compressed_size): + start_download_manager() + self.start() + value = self.cache.get(uuid) + if value is not None: + return value + blocks = _download_manager.get_blocks(uuid) + if blocks is None: + blocks = self.fetch_blocks(uuid, compressed_size) + value = self.from_blocks(uuid, blocks) + return value + + @staticmethod + def _get_blocks_by_filename(file_name, block_map): + fp = open(file_name, 'rb') + buf = fp.read() + blocks = [buf[offset: offset + size] for offset, size in block_map] + fp.close() + return blocks + + def fetch_blocks(self, uuid, compressed_size): + if uuid in self.shared_uuid_fn_dict: + return self._get_blocks_by_filename(self.shared_uuid_fn_dict[uuid], + self.shared_uuid_map_dict[uuid]) + download_sock = self.ctx.socket(zmq.REQ) + download_sock.connect(self.download_addr) + download_sock.send_pyobj((DATA_GET, + (uuid, compressed_size))) + res = download_sock.recv_pyobj() + if res == DATA_GET_OK: + return self._get_blocks_by_filename(self.shared_uuid_fn_dict[uuid], + self.shared_uuid_map_dict[uuid]) + if res == DATA_GET_FAIL: + raise RuntimeError('Data GET failed for uuid:%s' % uuid) + while True: + with self.download_cond: + if uuid not in self.shared_uuid_fn_dict: + self.download_cond.wait() else: - logger.error('Unknown server message: %s %s', type, msg) + break + if uuid in self.shared_uuid_fn_dict: + return self._get_blocks_by_filename(self.shared_uuid_fn_dict[uuid], + self.shared_uuid_map_dict[uuid]) + else: + raise RuntimeError('get blocks failed') - sock.close() - logger.debug("stop Broadcast server %s", server_addr) - for uuid in self.published.keys(): - self.clear(uuid) + def to_blocks(self, uuid, obj): + try: + if marshalable(obj): + buf = marshal.dumps((uuid, obj)) + type_ = MARSHAL_TYPE + else: + buf = cPickle.dumps((uuid, obj), -1) + type_ = PICKLE_TYPE - return server_addr, spawn(run) + except Exception: + buf = cPickle.dumps((uuid, obj), -1) + type_ = PICKLE_TYPE + + checksum = binascii.crc32(buf) & 0xFFFF + stream = struct.pack(self.header_fmt, type_, checksum) + buf + blockNum = (len(stream) + (BLOCK_SIZE - 1)) >> BLOCK_SHIFT + blocks = [compress(stream[i * BLOCK_SIZE:(i + 1) * BLOCK_SIZE]) for i in range(blockNum)] + sizes = [len(block) for block in blocks] + size_l = accumulate_list(sizes) + block_map = list(izip(size_l[:-1], sizes)) + return blocks, size_l[-1], block_map + + def from_blocks(self, uuid, blocks): + stream = b''.join(map(decompress, blocks)) + type_, checksum = struct.unpack(self.header_fmt, stream[:self.header_len]) + buf = stream[self.header_len:] + _checksum = binascii.crc32(buf) & 0xFFFF + if _checksum != checksum: + raise RuntimeError('Wrong blocks: checksum: %s, expected: %s' % ( + _checksum, checksum)) + + if type_ == MARSHAL_TYPE: + _uuid, value = marshal.loads(buf) + elif type_ == PICKLE_TYPE: + _uuid, value = cPickle.loads(buf) + else: + raise RuntimeError('Unknown serialization type: %s' % type_) + + if uuid != _uuid: + raise RuntimeError('Wrong blocks: uuid: %s, expected: %s' % (_uuid, uuid)) + + return value - def stop_server(self, addr): - req = env.ctx.socket(zmq.REQ) - req.setsockopt(zmq.LINGER, 0) - req.connect(addr) - req.send_pyobj((SERVER_STOP, None)) - avail = req.poll(1 * 100, zmq.POLLIN) - if avail: - req.recv_pyobj() - req.close() + def shutdown(self): + if not self._started: + return + + self._started = False + + +_manager = BroadcastManager() +_download_manager = DownloadManager() +_guide_manager = GuideManager() + + +def start_guide_manager(): + _guide_manager.start() -_manager = P2PBroadcastManager() +def start_download_manager(): + _download_manager.start() -def start_manager(is_master): - _manager.start(is_master) def stop_manager(): _manager.shutdown() + _download_manager.shutdown() + _guide_manager.shutdown() + env.environ.pop(GUIDE_ADDR, None) + env.environ.pop(DOWNLOAD_ADDR, None) -class Broadcast: + +class Broadcast(object): def __init__(self, value): assert value is not None, 'broadcast object should not been None' - self.uuid = str(uuid.uuid4()) + self.uuid = str(uuid_pkg.uuid4()) self.value = value - self.block_num = _manager.register(self.uuid, self.value) - self.bytes = self.block_num * BLOCK_SIZE - logger.info("broadcast %s in %d blocks", self.uuid, self.block_num) + self.compressed_size = _manager.register(self.uuid, self.value) + block_num = (self.compressed_size + BLOCK_SIZE - 1) >> BLOCK_SHIFT + self.bytes = block_num * BLOCK_SIZE + logger.info("broadcast %s in %d blocks, %d bytes", self.uuid, block_num, self.compressed_size) def clear(self): _manager.clear(self.uuid) def __getstate__(self): - return (self.uuid, self.block_num) + return self.uuid, self.compressed_size def __setstate__(self, v): - self.uuid, self.block_num = v + self.uuid, self.compressed_size = v def __getattr__(self, name): if name != 'value': return getattr(self.value, name) - value = _manager.fetch(self.uuid, self.block_num) + t = time.time() + value = _manager.fetch(self.uuid, self.compressed_size) if value is None: raise RuntimeError("fetch broadcast failed") + env.task_stats.secs_broadcast += time.time() - t self.value = value return value + def __len__(self): + return len(self.value) + + def __iter__(self): + return self.value.__iter__() + + def __getitem__(self, key): + return self.value.__getitem__(key) + + def __contains__(self, item): + return self.value.__contains__(item) + + def __missing__(self, key): + return self.value.__missing__(key) + + def __reversed__(self): + return self.value.__reversed__() diff --git a/dpark/cache.py b/dpark/cache.py index c0626888..af0c828e 100644 --- a/dpark/cache.py +++ b/dpark/cache.py @@ -1,20 +1,26 @@ +from __future__ import absolute_import +from __future__ import print_function import os -import multiprocessing -import logging import marshal -import cPickle +from six.moves import cPickle import shutil import struct -import urllib import msgpack from dpark.env import env +from dpark.serialize import marshalable +from dpark.utils import mkdir_p, atomic_file +from dpark.utils.log import get_logger from dpark.tracker import GetValueMessage, AddItemMessage, RemoveItemMessage +from six.moves import map +from six.moves import range +from six.moves import urllib -logger = logging.getLogger(__name__) +logger = get_logger(__name__) -class Cache: + +class Cache(object): data = {} def get(self, key): @@ -32,11 +38,13 @@ def put(self, key, value, is_iterator=False): def clear(self): self.data.clear() + class DiskCache(Cache): def __init__(self, tracker, path): - if not os.path.exists(path): - try: os.makedirs(path) - except: pass + try: + mkdir_p(path) + except: + pass self.tracker = tracker self.root = path @@ -58,7 +66,11 @@ def get(self, key): serve_uri = locs[-1] uri = '%s/cache/%s' % (serve_uri, os.path.basename(p)) - f = urllib.urlopen(uri) + try: + f = urllib.request.urlopen(uri) + except IOError: + logger.warning('urlopen cache uri %s failed', uri) + raise if f.code == 404: logger.warning('load from cache %s failed', uri) self.tracker.removeHost(rdd_id, index, serve_uri) @@ -76,15 +88,15 @@ def put(self, key, value, is_iterator=False): def clear(self): try: shutil.rmtree(self.root) - except OSError, e: + except OSError as e: pass def load(self, f): count, = struct.unpack("I", f.read(4)) if not count: return unpacker = msgpack.Unpacker(f, use_list=False) - for i in xrange(count): - _type, data = unpacker.next() + for i in range(count): + _type, data = next(unpacker) if _type == 0: yield marshal.loads(data) else: @@ -93,18 +105,18 @@ def load(self, f): def save(self, path, items): # TODO: purge old cache - tp = "%s.%d" % (path, os.getpid()) - with open(tp, 'wb') as f: + with atomic_file(path) as f: c = 0 f.write(struct.pack("I", c)) - try_marshal = True + # check is marshalable and compatible with broadcast + can_marshal = marshalable(items) for v in items: - if try_marshal: + if can_marshal: try: r = 0, marshal.dumps(v) except Exception: r = 1, cPickle.dumps(v, -1) - try_marshal = False + can_marshal = False else: r = 1, cPickle.dumps(v, -1) f.write(msgpack.packb(r)) @@ -112,26 +124,16 @@ def save(self, path, items): yield v bytes = f.tell() - if bytes > 10<<20: - logger.warning("cached result is %dMB (larger than 10MB)", bytes>>20) + if bytes > 10 << 20: + logger.warning("cached result is %dMB (larger than 10MB)", bytes >> 20) # count f.seek(0) f.write(struct.pack("I", c)) - os.rename(tp, path) class BaseCacheTracker(object): cache = None - def registerRDD(self, rddId, numPartitions): - pass - - def getLocationsSnapshot(self): - pass - - def getCachedLocs(self, rdd_id, index): - pass - def getCacheUri(self, rdd_id, index): pass @@ -164,33 +166,12 @@ def getOrCompute(self, rdd, split): def stop(self): self.clear() + class CacheTracker(BaseCacheTracker): def __init__(self): - cachedir = os.path.join(env.get('WORKDIR')[0], 'cache') + cachedir = env.workdir.get_path('cache') self.cache = DiskCache(self, cachedir) self.client = env.trackerClient - if env.isMaster: - self.locs = env.trackerServer.locs - self.rdds = {} - - def registerRDD(self, rddId, numPartitions): - self.rdds[rddId] = numPartitions - - def getLocationsSnapshot(self): - result = {} - for rdd_id, partitions in self.rdds.items(): - result[rdd_id] = [self.locs.get('cache:%s-%s' % (rdd_id, index), []) - for index in xrange(partitions)] - - return result - - def getCachedLocs(self, rdd_id, index): - def parse_hostname(uri): - if uri.startswith('http://'): - h = uri.split(':')[1].rsplit('/', 1)[-1] - return h - return '' - return map(parse_hostname, self.locs.get('cache:%s-%s' % (rdd_id, index), [])) def getCacheUri(self, rdd_id, index): return self.client.call(GetValueMessage('cache:%s-%s' % (rdd_id, index))) @@ -222,19 +203,31 @@ def __getstate__(self): raise Exception("!!!") -def test(): - logging.basicConfig(level=logging.DEBUG) - from dpark.context import DparkContext - dc = DparkContext("local") - dc.start() - nums = dc.parallelize(range(100), 10) - tracker = CacheTracker(True) - tracker.registerRDD(nums.id, len(nums)) - split = nums.splits[0] - print list(tracker.getOrCompute(nums, split)) - print list(tracker.getOrCompute(nums, split)) - print tracker.getLocationsSnapshot() - tracker.stop() - -if __name__ == '__main__': - test() +class CacheTrackerServer(object): + + def __init__(self): + self.locs = env.trackerServer.locs + self.rdds = {} + + def registerRDD(self, rddId, numPartitions): + self.rdds[rddId] = numPartitions + + def getCachedLocs(self, rdd_id, index): + def parse_hostname(uri): + if uri.startswith('http://'): + h = uri.split(':')[1].rsplit('/', 1)[-1] + return h + return '' + + return list(map(parse_hostname, self.locs.get('cache:%s-%s' % (rdd_id, index), []))) + + def getLocationsSnapshot(self): + result = {} + for rdd_id, partitions in self.rdds.items(): + result[rdd_id] = [self.locs.get('cache:%s-%s' % (rdd_id, index), []) + for index in range(partitions)] + + return result + + def __getstate__(self): + raise Exception("!!!") diff --git a/dpark/conf.py b/dpark/conf.py index 4f3b8fc7..68630de4 100644 --- a/dpark/conf.py +++ b/dpark/conf.py @@ -1,7 +1,10 @@ -import os.path -import logging +from __future__ import absolute_import +import os +from dpark.utils.log import get_logger -logger = logging.getLogger(__name__) +# override configs use python file at path given by env var $DPARK_CONF, see the end of this file + +logger = get_logger(__name__) # workdir used in slaves for internal files # @@ -11,25 +14,137 @@ # uri of mesos master, host[:5050] or or zk://... MESOS_MASTER = 'localhost' +MESOS_MASTERS = {} + +# used for mrun, allow it to get all resources quickly, maybe more than a fair share. +MESOS_MPI_ROLE = "mpi" # mount points of MooseFS, must be available on all slaves # for example: '/mfs' : 'mfsmaster', MOOSEFS_MOUNT_POINTS = { } + +# dup log to path lick $LOGHUB/$LOGHUB_PATH_FORMAT/$FRAMEWORK_ID/log +# $LOGHUB/$LOGHUB_PATH_FORMAT should exists before run +LOGHUB = None +LOGHUB_PATH_FORMAT = "%Y/%m/%d/%H" + +ENABLE_ES_LOGHUB = False +ES_HOST = None +ES_INDEX = None +ES_TYPE = None + # consistant dir cache in client, need patched mfsmaster MOOSEFS_DIR_CACHE = False # memory used per task, like -M (--m) option in context. MEM_PER_TASK = 200.0 +MAX_OPEN_FILE = 900 +LOG_ROTATE = True +MULTI_SEGMENT_DUMP = True + +TIME_TO_SUPPRESS = 60 # sec + + +OP_UDF = "udf" +OP_GROUPBY = "groupby" +OP_COGROUP = "cogroup" + +DEFAULT_TASK_TIME = 3600 # 1 hour + +_named_only_start = object() + + +class RDDConf(object): + # use default_rddconf to set default values, do NOT change ATTRS + ATTRS = { + "disk_merge": False, + "sort_merge": False, + "iter_group": False, + "ordered_group": False, + "dump_mem_ratio": 0.9, + "op": OP_UDF, + "_dummy": _named_only_start, + } + + def __init__(self, _dummy, disk_merge, sort_merge, iter_group, ordered_group, dump_mem_ratio, op): + if _dummy != _named_only_start: + raise TypeError("DO NOT use RDDConf directly; use dpark.conf.rddconf() instead. ") + + self.disk_merge = disk_merge + self.sort_merge = sort_merge + self.iter_group = iter_group + self.ordered_group = ordered_group + self.dump_mem_ratio = dump_mem_ratio + self.op = op + + def __setattr__(self, name, value): + if name not in self.ATTRS: + msg = "'RDDConf' object has no attribute '{}'. Valid attrs: {}".format(name, self.to_dict().keys()) + raise AttributeError(msg) + object.__setattr__(self, name, value) + + def to_dict(self): + d = dict(self.__dict__) + return d + + def __repr__(self): + return "RDDConf_%r" % (self.to_dict()) + + def dup(self, **kwargs): + res = RDDConf(_named_only_start, **self.__dict__) + for k, v in kwargs.items(): + res.__setattr__(k, v) + return res + + @property + def is_cogroup(self): + return self.op == OP_COGROUP + + @property + def is_groupby(self): + return self.op == OP_GROUPBY + + +default_rddconf = RDDConf(**RDDConf.ATTRS) # for user +_rdd = default_rddconf # for etc + + +def rddconf(_dummy=_named_only_start, + disk_merge=None, sort_merge=None, + iter_group=False, ordered_group=None, + dump_mem_ratio=None, + op=OP_UDF): + """ Return new RDDConfig object based on default values. + Only takes named arguments. + e.g. groupByKey(.., rddconf=dpark.conf.rddconf(...)) + """ + if _dummy != _named_only_start: + raise TypeError("rddconf() only takes named arguments") + kwargs = locals() + kwargs.pop("_dummy") + kwargs = dict([kv for kv in kwargs.items() if kv[1] is not None]) + res = default_rddconf.dup(**kwargs) + return res + + +def ban(hostname): + return False + + def load_conf(path): if not os.path.exists(path): - logger.error("conf %s do not exists", path) - raise Exception("conf %s do not exists" % path) + logger.warning("conf %s do not exists, use default config", path) + return try: - data = open(path).read() - exec data in globals(), globals() - except Exception, e: + with open(path) as f: + data = f.read() + exec(data, globals(), globals()) + except Exception as e: logger.error("error while load conf from %s: %s", path, e) raise + + +load_conf(os.environ.get('DPARK_CONF', '/etc/dpark.conf')) diff --git a/dpark/context.py b/dpark/context.py index 69510558..0f137c30 100644 --- a/dpark/context.py +++ b/dpark/context.py @@ -1,4 +1,4 @@ -import os, sys +from __future__ import absolute_import import atexit import optparse import signal @@ -6,80 +6,177 @@ import gc from dpark.rdd import * +from dpark.utils.beansdb import restore_value from dpark.accumulator import Accumulator -from dpark.schedule import LocalScheduler, MultiProcessScheduler, MesosScheduler +from dpark.schedule import ( + LocalScheduler, MultiProcessScheduler, MesosScheduler +) from dpark.env import env -from dpark.moosefs import walk +from dpark.file_manager import walk from dpark.tabular import TabularRDD +from dpark.utils import memory_str_to_mb +from dpark.utils.log import init_dpark_logger, get_logger +from dpark.utils.debug import spawn_rconsole import dpark.conf as conf from math import ceil +import socket +from six.moves import range + +logger = get_logger(__name__) + + +def is_gevent_monkey_patched(): + try: + from gevent import monkey + except ImportError: + return False + else: + return bool(getattr(monkey, 'saved', False)) -logger = logging.getLogger(__name__) def singleton(cls): instances = {} + def getinstance(*a, **kw): key = (cls, tuple(a), tuple(sorted(kw.items()))) if key not in instances: + if len(instances) >= 1: + logger.error('Too many DparkContext created!') + instances[key] = cls(*a, **kw) + return instances[key] + + getinstance._instances = instances return getinstance -def setup_conf(options): - if options.conf: - conf.load_conf(options.conf) - elif 'DPARK_CONF' in os.environ: - conf.load_conf(os.environ['DPARK_CONF']) - elif os.path.exists('/etc/dpark.conf'): - conf.load_conf('/etc/dpark.conf') - if options.mem is None: - options.mem = conf.MEM_PER_TASK +_shutdown_handlers = [] + + +def _shutdown(): + started = len(_shutdown_handlers) + for handler in _shutdown_handlers: + try: + handler() + except Exception: + logger.exception('Faield to shutdown context') + + _shutdown_handlers[:] = [] + DparkContext._instances.clear() + if started: + logger.info("dpark shutdown.") + + +atexit.register(_shutdown) +_pid = os.getpid() +_prev_handlers = {} +_signals = [ + signal.SIGINT, signal.SIGQUIT, signal.SIGTERM, + signal.SIGABRT, signal.SIGHUP +] + + +def _handler(signum, frame): + for sig, handler in _prev_handlers.items(): + signal.signal(sig, handler) + + if _pid == os.getpid(): + # called on main process, will do cleanup + logger.error("got signal %d, exiting now...", signum) + _shutdown() + + # resend signal to trigger previous handlers + _prev_handlers.clear() + os.kill(os.getpid(), signum) + + +def register_sighandlers(): + if not _prev_handlers: + for sig in _signals: + try: + _prev_handlers[sig] = signal.signal(sig, _handler) + except Exception: + logger.exception('Failed to register signal handler') - conf.__dict__.update(os.environ) - import moosefs - moosefs.MFS_PREFIX = conf.MOOSEFS_MOUNT_POINTS - moosefs.master.ENABLE_DCACHE = conf.MOOSEFS_DIR_CACHE @singleton class DparkContext(object): nextShuffleId = 0 + options = None + def __init__(self, master=None): + if is_gevent_monkey_patched(): + raise RuntimeError('DPark do not support gevent.') + self.master = master self.initialized = False self.started = False + self.web_port = None + self.webui_url = None + self.data_limit = None + self.scheduler = None + self.is_local = True self.defaultParallelism = 2 + self.defaultMinSplits = 2 + self.is_dstream = False def init(self): if self.initialized: return - options = parse_options() - self.options = options + register_sighandlers() - master = self.master or options.master + cls = self.__class__ + options = cls.options + if options is None: + options = cls.options = parse_options() + + try: + import dpark.web + from dpark.web.ui import create_app + app = create_app(self) + self.web_port = dpark.web.start(app) + self.webui_url = 'http://%s:%s' % ( + socket.gethostname(), + self.web_port + ) + logger.info('start listening on Web UI: %s', self.webui_url) + except ImportError as e: + self.webui_url = None + logger.info('no web server created as %s', e) + + origin_master = master = self.master or options.master if master == 'local': + logger.info("use local scheduler: %s", master) self.scheduler = LocalScheduler() - self.isLocal = True + self.is_local = True elif master == 'process': + logger.info("use process scheduler: %s", master) self.scheduler = MultiProcessScheduler(options.parallel) - self.isLocal = False + self.is_local = False else: if master == 'mesos': master = conf.MESOS_MASTER + else: + master = conf.MESOS_MASTERS.get(master, master) if master.startswith('mesos://'): if '@' in master: - master = master[master.rfind('@')+1:] + master = master[master.rfind('@') + 1:] else: - master = master[master.rfind('//')+2:] + master = master[master.rfind('//') + 2:] elif master.startswith('zoo://'): master = 'zk' + master[3:] if ':' not in master: master += ':5050' - self.scheduler = MesosScheduler(master, options) - self.isLocal = False + self.scheduler = MesosScheduler( + master, options, webui_url=self.webui_url + ) + self.data_limit = 1024 * 1024 # 1MB + self.is_local = False + logger.info("use mesos scheduler: %s", master) self.master = master @@ -90,10 +187,15 @@ def init(self): self.defaultMinSplits = max(self.defaultParallelism, 2) self.initialized = True + self.scheduler.is_dstream = self.is_dstream + + logger.info("DparkContext initialized, use master %s -> %s, default_rddconf = %s", + origin_master, master, + conf.default_rddconf) @staticmethod def setLogLevel(level): - logging.getLogger('dpark').setLevel(level) + get_logger('dpark').setLevel(level) def newShuffleId(self): self.nextShuffleId += 1 @@ -112,22 +214,23 @@ def textFile(self, path, ext='', followLink=True, maxdepth=0, cls=TextFileRDD, * self.init() if isinstance(path, (list, tuple)): return self.union([self.textFile(p, ext, followLink, maxdepth, cls, *ka, **kws) - for p in path]) + for p in path]) path = os.path.realpath(path) - def create_rdd(cls, path, *ka, **kw): - if cls is TextFileRDD: - if path.endswith('.bz2'): - return BZip2FileRDD(self, path, *ka, **kw) - elif path.endswith('.gz'): - return GZipFileRDD(self, path, *ka, **kw) - return cls(self, path, *ka, **kw) + + def create_rdd(_cls, _path, *_ka, **_kw): + if _cls is TextFileRDD: + if _path.endswith('.bz2'): + return BZip2FileRDD(self, _path, *_ka, **_kw) + elif _path.endswith('.gz'): + return GZipFileRDD(self, _path, *_ka, **_kw) + return _cls(self, _path, *_ka, **_kw) if os.path.isdir(path): paths = [] - for root,dirs,names in walk(path, followlinks=followLink): + for root, dirs, names in walk(path, followlinks=followLink): if maxdepth > 0: - depth = len(filter(None, root[len(path):].split('/'))) + 1 + depth = len([_f for _f in root[len(path):].split('/') if _f]) + 1 if depth > maxdepth: break for n in sorted(names): @@ -141,17 +244,20 @@ def create_rdd(cls, path, *ka, **kw): dirs.remove(d) rdds = [create_rdd(cls, p, *ka, **kws) - for p in paths] + for p in paths] return self.union(rdds) else: return create_rdd(cls, path, *ka, **kws) + def tfRecordsFile(self, path, *args, **kwargs): + return self.textFile(path, cls=TfrecordsRDD, *args, **kwargs) + def partialTextFile(self, path, begin, end, splitSize=None, numSplits=None): self.init() return PartialTextFileRDD(self, path, begin, end, splitSize, numSplits) def bzip2File(self, *args, **kwargs): - "deprecated" + """deprecated""" logger.warning("bzip2File() is deprecated, use textFile('xx.bz2') instead") return self.textFile(cls=BZip2FileRDD, *args, **kwargs) @@ -173,41 +279,80 @@ def table(self, path, **kwargs): for root, dirs, names in walk(dpath): if '.field_names' in names: p = os.path.join(root, '.field_names') - fields = open(p).read().split('\t') + with open(p) as f: + fields = f.read().split('\t') + break else: raise Exception("no .field_names found in %s" % path) return self.tableFile(path, **kwargs).asTable(fields) - def beansdb(self, path, depth=None, filter=None, fullscan=False, raw=False, only_latest=False): - "(Key, (Value, Version, Timestamp)) data in beansdb" + def beansdb(self, path, depth=None, filter=None, + fullscan=False, raw=False, only_latest=False): + """(Key, (VALUE, Version, Timestamp)) data in beansdb + + Data structure: + REC = (Key, TRIPLE) + TRIPLE = (VALUE, Version, Timestamp) + VALUE = RAW_VALUE | REAL_VALUE + RAW_VALUE = (flag, BYTES_VALUE) + + Args: + path: beansdb data path + filter: used to filter key + depth: choice = [None, 0, 1, 2]. e.g. depth=2 assume dir tree like: + 'path/[0-F]/[0-F]/%03d.data' + If depth is None, dpark will guess. + fullscan: NOT use index files, which contain (key, pos_in_datafile). + pairs. + Better use fullscan unless the filter selectivity is low. + Effect of using index: + inefficient random access + one split(task) for each file instead of each moosefs chunk + + Omitted if filter is None. + raw: VALUE = RAW_VALUE if raw else REAL_VALUE. + only_latest: for each key, keeping the REC with the largest + Timestamp. This will append a reduceByKey RDD. + Need this because online beansdb data is log structured. + """ + + key_filter = filter + self.init() + if key_filter is None: + fullscan = True if isinstance(path, (tuple, list)): - return self.union([self.beansdb(p, depth, filter, fullscan, raw, only_latest) - for p in path]) - - path = os.path.realpath(path) - assert os.path.exists(path), "%s no exists" % path - if os.path.isdir(path): - subs = [] - if not depth: - subs = [os.path.join(path, n) for n in os.listdir(path) if n.endswith('.data')] - if subs: - rdd = self.union([BeansdbFileRDD(self, p, filter, fullscan, True) - for p in subs]) - else: - subs = [os.path.join(path, '%x'%i) for i in range(16)] - rdd = self.union([self.beansdb(p, depth and depth-1, filter, fullscan, True, only_latest) - for p in subs if os.path.exists(p)]) - only_latest = False + rdd = self.union([self.beansdb(p, depth, key_filter, fullscan, + raw=True, only_latest=False) + for p in path]) else: - rdd = BeansdbFileRDD(self, path, filter, fullscan, True) + path = os.path.realpath(path) + assert os.path.exists(path), "%s no exists" % path + if os.path.isdir(path): + subs = [] + if not depth: + subs = [os.path.join(path, n) for n in os.listdir(path) + if n.endswith('.data')] + if subs: + rdd = self.union([BeansdbFileRDD(self, p, key_filter, + fullscan, raw=True) + for p in subs]) + else: + subs = [os.path.join(path, '%x' % i) for i in range(16)] + rdd = self.union([self.beansdb(p, depth and depth - 1, key_filter, + fullscan, raw=True, only_latest=False) + for p in subs if os.path.exists(p)]) + else: + rdd = BeansdbFileRDD(self, path, key_filter, fullscan, raw) # choose only latest version if only_latest: - rdd = rdd.reduceByKey(lambda v1,v2: v1[2] > v2[2] and v1 or v2, int(ceil(len(rdd) / 4))) + num_splits = min(int(ceil(len(rdd) / 4)), 800) + rdd = rdd.reduceByKey(lambda v1, v2: v1[2] > v2[2] and v1 or v2, + numSplits=num_splits) if not raw: - rdd = rdd.mapValue(lambda (v,ver,t): (restore_value(*v), ver, t)) + rdd = rdd.mapValue(lambda v_ver_t: (restore_value(*v_ver_t[0]), v_ver_t[1], v_ver_t[2])) return rdd def union(self, rdds): @@ -225,42 +370,43 @@ def broadcast(self, v): return Broadcast(v) def start(self): + def shutdown(): + self.stop() + try: + import dpark.web + dpark.web.stop(self.web_port) + except ImportError: + pass + if self.started: return self.init() - env.start(True) + env.start_master() + if not isinstance(self.scheduler, MesosScheduler): + env.start_slave() + self.scheduler.start() self.started = True - atexit.register(self.stop) + _shutdown_handlers.append(shutdown) - def handler(signm, frame): - logger.error("got signal %d, exit now", signm) - self.scheduler.shutdown() - try: - signal.signal(signal.SIGTERM, handler) - signal.signal(signal.SIGHUP, handler) - signal.signal(signal.SIGABRT, handler) - signal.signal(signal.SIGQUIT, handler) - except: pass - - try: - from rfoo.utils import rconsole - rconsole.spawn_server(locals(), 0) - except ImportError: - pass + spawn_rconsole(locals()) def runJob(self, rdd, func, partitions=None, allowLocal=False): self.start() + success = False if partitions is None: - partitions = range(len(rdd)) + partitions = list(range(len(rdd))) try: gc.disable() for it in self.scheduler.runJob(rdd, func, partitions, allowLocal): yield it + success = True finally: + if not success: + logger.critical('Framework failed') gc.collect() gc.enable() @@ -283,65 +429,91 @@ def __getstate__(self): raise ValueError("should not pickle ctx") -parser = optparse.OptionParser(usage="Usage: %prog [options] [args]") +class Parser(optparse.OptionParser): + def _process_args(self, largs, rargs, values): + while rargs: + arg = rargs[0] + if arg == "--": + del rargs[0] + return + elif arg[0:2] == "--": + try: + self._process_long_opt(rargs, values) + except optparse.AmbiguousOptionError: + raise + except optparse.BadOptionError: + largs.append(arg) + return + elif arg[:1] == "-" and len(arg) > 1: + try: + self._process_short_opts(rargs, values) + except optparse.AmbiguousOptionError: + raise + except optparse.BadOptionError: + largs.append(arg) + return + else: + return + + +parser = Parser(usage="Usage: %prog [options] [args]") -def add_default_options(): - parser.disable_interspersed_args() +def add_default_options(): group = optparse.OptionGroup(parser, "Dpark Options") group.add_option("-m", "--master", type="string", default="local", - help="master of Mesos: local, process, host[:port], or mesos://") -# group.add_option("-n", "--name", type="string", default="dpark", -# help="job name") + help="master of Mesos: local, process, host[:port], or mesos://") group.add_option("-p", "--parallel", type="int", default=0, - help="number of processes") + help="number of processes") group.add_option("-c", "--cpus", type="float", default=1.0, - help="cpus used per task") - group.add_option("-M", "--mem", type="float", - help="memory used per task") + help="cpus used per task") + group.add_option("-M", "--mem", type="string", + help="memory used per task (e.g. 300m, 1g), default unit is 'm'") group.add_option("-g", "--group", type="string", default="", - help="which group of machines") + help="which group of machines") group.add_option("--err", type="float", default=0.0, - help="acceptable ignored error record ratio (0.01%)") - group.add_option("--snapshot_dir", type="string", default="", - help="shared dir to keep snapshot of RDDs") - - group.add_option("--conf", type="string", - help="path for configuration file") - group.add_option("--self", action="store_true", - help="user self as exectuor") + help="acceptable ignored error record ratio (0.01%)") + group.add_option("--checkpoint_dir", type="string", default="", + help="shared dir to keep checkpoint of RDDs") + + group.add_option("--color", action="store_true") + group.add_option("--no-color", action="store_false", dest='color') + group.add_option("--profile", action="store_true", - help="do profiling") - group.add_option("--keep-order", action="store_true", - help="deprecated, always keep order") + help="do profiling") + group.add_option("--role", type="string", default="") + + group.add_option("-I", "--image", type="string", + help="image name for Docker") + group.add_option("-V", "--volumes", type="string", + help="volumes to mount into Docker") parser.add_option_group(group) parser.add_option("-q", "--quiet", action="store_true") parser.add_option("-v", "--verbose", action="store_true") + add_default_options() def parse_options(): options, args = parser.parse_args() - setup_conf(options) + if options.mem is None: + options.mem = conf.MEM_PER_TASK + else: + options.mem = memory_str_to_mb(options.mem) options.logLevel = (options.quiet and logging.ERROR - or options.verbose and logging.DEBUG or logging.INFO) - - log_format = '%(asctime)-15s [%(levelname)s] [%(name)-9s] %(message)s' - logging.basicConfig(format=log_format, level=options.logLevel) - - logger = logging.getLogger('dpark') - logger.propagate=False + or options.verbose and logging.DEBUG or logging.INFO) + if options.color is None: + options.color = getattr(sys.stderr, 'isatty', lambda: False)() - handler = logging.StreamHandler() - handler.setFormatter(logging.Formatter(log_format)) + init_dpark_logger(options.logLevel, use_color=options.color) - logger.addHandler(handler) - logger.setLevel(max(options.logLevel, logger.level)) + if any(arg.startswith('-') for arg in args): + logger.warning('unknown args found in command-line: %s', ' '.join(args)) return options diff --git a/dpark/decorator.py b/dpark/decorator.py deleted file mode 100644 index 4ef1565a..00000000 --- a/dpark/decorator.py +++ /dev/null @@ -1,35 +0,0 @@ -# -*- coding: utf-8 -*- -class LazyJIT(object): - this = None - def __init__(self, decorator, f, *args, **kwargs): - self.f = f - self.args = args - self.kwargs = kwargs - self.decorator = decorator - - def __call__(self, *args, **kwargs): - if self.this is None: - try: - mod = __import__('numba', fromlist=[self.decorator]) - d = getattr(mod, self.decorator) - self.this = d(*self.args, **self.kwargs)(self.f) - except ImportError, e: - self.this = self.f - return getattr(self.this, '__call__')(*args, **kwargs) - - -def jit(signature, **kwargs): - if not isinstance(signature, (str, unicode)): - raise ValueError('First argument should be signature') - def _(f): - return LazyJIT('jit', f, signature, **kwargs) - return _ - -def autojit(*args, **kwargs): - if len(args) ==1 and not kwargs and callable(args[0]): - f = args[0] - return LazyJIT('autojit', f) - else: - def _(f): - return LazyJIT('autojit', f, *args, **kwargs) - return _ diff --git a/dpark/dependency.py b/dpark/dependency.py index 7794f59b..e454baa0 100644 --- a/dpark/dependency.py +++ b/dpark/dependency.py @@ -1,7 +1,11 @@ +from __future__ import absolute_import import bisect -from dpark.util import portable_hash +from dpark.utils import portable_hash from dpark.serialize import load_func, dump_func +from dpark.utils.heaponkey import HeapOnKey +from six.moves import range + class Dependency: def __init__(self, rdd): @@ -10,15 +14,19 @@ def __init__(self, rdd): def __getstate__(self): raise ValueError("Should not pickle dependency: %r" % self) + class NarrowDependency(Dependency): isShuffle = False + def getParents(self, outputPartition): raise NotImplementedError + class OneToOneDependency(NarrowDependency): def getParents(self, pid): return [pid] + class OneToRangeDependency(NarrowDependency): def __init__(self, rdd, splitSize, length): Dependency.__init__(self, rdd) @@ -26,8 +34,9 @@ def __init__(self, rdd, splitSize, length): self.length = length def getParents(self, pid): - return range(pid * self.splitSize, - min((pid+1) * self.splitSize, self.length)) + return list(range(pid * self.splitSize, + min((pid + 1) * self.splitSize, self.length))) + class CartesianDependency(NarrowDependency): def __init__(self, rdd, first, numSplitsInRdd2): @@ -41,6 +50,7 @@ def getParents(self, pid): else: return [pid % self.numSplitsInRdd2] + class RangeDependency(NarrowDependency): def __init__(self, rdd, inStart, outStart, length): Dependency.__init__(self, rdd) @@ -49,30 +59,76 @@ def __init__(self, rdd, inStart, outStart, length): self.length = length def getParents(self, pid): - if pid >= self.outStart and pid < self.outStart + self.length: + if self.outStart <= pid < self.outStart + self.length: return [pid - self.outStart + self.inStart] return [] + class ShuffleDependency(Dependency): isShuffle = True - def __init__(self, shuffleId, rdd, aggregator, partitioner): + + def __init__(self, shuffleId, rdd, aggregator, partitioner, rddconf): Dependency.__init__(self, rdd) self.shuffleId = shuffleId self.aggregator = aggregator self.partitioner = partitioner + self.rddconf = rddconf + + +class AggregatorBase(object): + + def createCombiner(self, x): + raise NotImplementedError(self.__class__.__name__) + + def mergeValue(self, s, x): + raise NotImplementedError(self.__class__.__name__) + def mergeCombiners(self, x, y): + raise NotImplementedError(self.__class__.__name__) + + def aggregate_sorted(self, items): + create = self.createCombiner + merge = self.mergeValue + i = None + for i, (k, v) in enumerate(items): + if i == 0: + curr_key = k + curr_value = create(v) + elif k != curr_key: + yield curr_key, curr_value + curr_key = k + curr_value = create(v) + else: + curr_value = merge(curr_value, v) + if i is not None: + yield curr_key, curr_value + + +class GroupByAggregator(AggregatorBase): + + def createCombiner(self, x): + return [x] + + def mergeValue(self, c, x): + c.append(x) + return c -class Aggregator: + def mergeCombiners(self, x, y): + x.extend(y) + return x + + +class Aggregator(object): def __init__(self, createCombiner, mergeValue, - mergeCombiners): + mergeCombiners): self.createCombiner = createCombiner self.mergeValue = mergeValue self.mergeCombiners = mergeCombiners def __getstate__(self): return (dump_func(self.createCombiner), - dump_func(self.mergeValue), - dump_func(self.mergeCombiners)) + dump_func(self.mergeValue), + dump_func(self.mergeCombiners)) def __setstate__(self, state): c1, c2, c3 = state @@ -80,57 +136,109 @@ def __setstate__(self, state): self.mergeValue = load_func(c2) self.mergeCombiners = load_func(c3) + class AddAggregator: def createCombiner(self, x): return x + def mergeValue(self, s, x): return s + x + def mergeCombiners(self, x, y): return x + y + class MergeAggregator: def createCombiner(self, x): return [x] + def mergeValue(self, s, x): s.append(x) return s + def mergeCombiners(self, x, y): x.extend(y) return x + +class HeapAggregator: + + def __init__(self, heap_limit, key=None, order_reverse=False): + self.heap = HeapOnKey(key=key, min_heap=order_reverse) + self.heap_limit = heap_limit + assert (heap_limit > 0) + + def __getstate__(self): + return self.heap, self.heap_limit + + def __setstate__(self, state): + self.heap, self.heap_limit = state + + def createCombiner(self, x): + return [x] + + def mergeValue(self, s, x): + if len(s) >= self.heap_limit: + self.heap.push_pop(s, x) + else: + self.heap.push(s, x) + return s + + def mergeCombiners(self, x, y): + for item in y: + if len(x) < self.heap_limit: + self.heap.push(x, item) + else: + self.heap.push_pop(x, item) + return x + + class UniqAggregator: def createCombiner(self, x): return set([x]) + def mergeValue(self, s, x): s.add(x) return s + def mergeCombiners(self, x, y): x |= y return x + class Partitioner: @property def numPartitions(self): raise NotImplementedError + def getPartition(self, key): raise NotImplementedError + class HashPartitioner(Partitioner): - def __init__(self, partitions): + def __init__(self, partitions, thresholds=None): self.partitions = max(1, int(partitions)) + self.thresholds = thresholds + assert self.partitions != 0 + assert self.thresholds is None or len(self.thresholds) == self.partitions - 1 @property def numPartitions(self): return self.partitions def getPartition(self, key): - return portable_hash(key) % self.partitions + if self.thresholds is None: + return portable_hash(key) % self.partitions + else: + return bisect.bisect(self.thresholds, portable_hash(key)) def __eq__(self, other): - if isinstance(other, Partitioner): - return other.numPartitions == self.numPartitions + if isinstance(other, HashPartitioner): + return other.numPartitions == self.numPartitions and \ + other.thresholds == self.thresholds return False + class RangePartitioner(Partitioner): def __init__(self, keys, reverse=False): self.keys = sorted(keys) diff --git a/dpark/dstream.py b/dpark/dstream.py index 1ac8bddf..ba2a7be8 100644 --- a/dpark/dstream.py +++ b/dpark/dstream.py @@ -1,44 +1,83 @@ +from __future__ import absolute_import +from __future__ import print_function import os import time -import math +import socket +import shutil import itertools -import logging +import threading import random - -from dpark.util import spawn +from functools import reduce +from collections import deque +from contextlib import closing +import six +from six.moves import range + +try: + import six.moves.cPickle as pickle +except ImportError: + import pickle + +from dpark.utils import spawn, atomic_file +from dpark.utils.log import get_logger +from dpark.serialize import load_func, dump_func from dpark.dependency import Partitioner, HashPartitioner, Aggregator from dpark.context import DparkContext -from dpark.rdd import CoGroupedRDD +from dpark.rdd import CoGroupedRDD, CheckpointRDD +from dpark.file_manager import open_file +from dpark.file_manager.utils import Error + +try: + from scribe.scribe import Iface, ResultCode, Processor + from thrift.protocol import TBinaryProtocol + from thrift.transport import TSocket + from thrift.server import TNonblockingServer + from collections import deque + from kazoo.client import KazooClient + + WITH_SCRIBE = True +except: + WITH_SCRIBE = False + +logger = get_logger(__name__) -logger = logging.getLogger(__name__) class Interval(object): + def __init__(self, beginTime, endTime): self.begin = beginTime self.end = endTime + @property def duration(self): return self.end - self.begin + def __add__(self, d): return Interval(self.begin + d, self.end + d) + def __sub__(self, d): return Interval(self.begin - d, self.end - d) + def __le__(self, that): assert self.duration == that.duration return self.begin < that.begin + def __ge__(self, that): return not self < that + def __str__(self): return '[%s, %s]' % (self.begin, self.end) + @classmethod def current(cls, duration): now = int(time.time()) ss = int(duration) - begin = now / ss * ss + begin = now // ss * ss return cls(begin, begin + duration) class DStreamGraph(object): + def __init__(self, batchDuration): self.inputStreams = [] self.outputStreams = [] @@ -47,9 +86,8 @@ def __init__(self, batchDuration): self.rememberDuration = None def start(self, time): - self.zeroTime = int(time / self.batchDuration) * self.batchDuration + self.zeroTime = int(time // self.batchDuration) * self.batchDuration for out in self.outputStreams: - #out.initialize(time) out.remember(self.rememberDuration) for ins in self.inputStreams: ins.start() @@ -58,9 +96,12 @@ def stop(self): for ins in self.inputStreams: ins.stop() - #def setContext(self, ssc): - # for out in self.outputStreams: - # out.setContext(ssc) + def setContext(self, ssc): + for ins in self.inputStreams: + ins.setContext(ssc) + + for out in self.outputStreams: + out.setContext(ssc) def remember(self, duration): self.rememberDuration = duration @@ -74,8 +115,8 @@ def addOutputStream(self, output): self.outputStreams.append(output) def generateRDDs(self, time): - #print 'generateRDDs', self, time - return filter(None, [out.generateJob(time) for out in self.outputStreams]) + return [_f for _f in [out.generateJob(time) + for out in self.outputStreams] if _f] def forgetOldRDDs(self, time): for out in self.outputStreams: @@ -85,47 +126,49 @@ def updateCheckpointData(self, time): for out in self.outputStreams: out.updateCheckpointData(time) - def restoreCheckpointData(self, time): + def restoreCheckpointData(self): for out in self.outputStreams: - out.restoreCheckpointData(time) + out.restoreCheckpointData() class StreamingContext(object): - def __init__(self, batchDuration, sc=None): - if isinstance(sc, str) or not sc: # None + + def __init__(self, batchDuration, sc=None, graph=None, batchCallback=None): + if isinstance(sc, str) or not sc: # None sc = DparkContext(sc) self.sc = sc + sc.is_dstream = True + batchDuration = int(batchDuration) self.batchDuration = batchDuration - self.graph = DStreamGraph(batchDuration) + self.graph = graph or DStreamGraph(batchDuration) self.checkpointDir = None self.checkpointDuration = None - self.networkInputTracker = None self.scheduler = None - self.receiverJobThread = None - - # def load(self, cp): - # if isinstance(cp, str): - # cp = Checkpoint.read(cp) - # self.cp = cp - # self.sc = DparkContext(cp.master) - # self.graph = cp.graph - # self.graph.setContext(self) - # self.graph.restoreCheckpointData() - # #self.sc.setCheckpointDir(cp.checkpointDir, True) - # self.checkpointDir = cp.checkpointDir - # self.checkpointDuration = cp.checkpointDuration + self.lastCheckpointTime = 0 + self.batchCallback = batchCallback + + @classmethod + def load(cls, path, sc=None): + cp = Checkpoint.read(path) + graph = cp.graph + if not sc: + sc = cp.master + ssc = cls(cp.batchDuration, sc, graph) + ssc.checkpointDir = path + ssc.checkpointDuration = cp.checkpointDuration + graph.setContext(ssc) + graph.restoreCheckpointData() + return ssc, cp.time def remember(self, duration): self.graph.remember(duration) def checkpoint(self, directory, interval): - #if directory: - # self.sc.setCheckpointDir(directory) self.checkpointDir = directory self.checkpointDuration = interval - #def getInitialCheckpoint(self): + # def getInitialCheckpoint(self): # return self.cp def registerInputStream(self, ds): @@ -139,13 +182,38 @@ def networkTextStream(self, hostname, port): self.registerInputStream(ds) return ds - def fileStream(self, directory, filter=None, newFilesOnly=True): - ds = FileInputDStream(self, directory, filter, newFilesOnly) + def scribeTextStream(self, zk_address, zk_path): + if WITH_SCRIBE: + ds = ScribeInputDStream(self, zk_address, zk_path, category='default') + self.registerInputStream(ds) + return ds + else: + raise RuntimeError('No scribed env supported') + + def customStream(self, func): + ds = NetworkInputDStream(self, func) + self.registerInputStream(ds) + return ds + + def fileStream(self, directory, filter=None, newFilesOnly=True, oldThreshold=3600): + """ + `oldThreshold`: If a file is not modified in last `oldThreshold` seconds, it will be + classified as an old file, and will be ommitted in future batches. The default value is + 3600, this may be **changed** in the future, we highly recommend user to set `oldThreshold` + explicitly. + """ + ds = FileInputDStream(self, directory, filter, newFilesOnly, oldThreshold) + self.registerInputStream(ds) + return ds + + def rotatingfiles(self, files): + ds = RotatingFilesInputDStream(self, files) self.registerInputStream(ds) return ds - def textFileStream(self, directory, filter=None, newFilesOnly=True): - return self.fileStream(directory, filter, newFilesOnly).map(lambda (k,v):v) + def textFileStream(self, directory, filter=None, newFilesOnly=True, oldThreshold=3600): + return self.fileStream( + directory, filter, newFilesOnly, oldThreshold).map(lambda k_v: k_v[1]) def makeStream(self, rdd): return ConstantInputDStream(self, rdd) @@ -159,53 +227,72 @@ def union(self, streams): return UnionDStream(streams) def start(self, t=None): - if not self.checkpointDuration: - self.checkpointDuration = self.batchDuration - - # TODO - #nis = [ds for ds in self.graph.inputStreams - # if isinstance(ds, NetworkInputDStream)] - #if nis: - # self.networkInputTracker = NetworkInputTracker(self, nis) - # self.networkInputTracker.start() - self.sc.start() + nis = [ds for ds in self.graph.inputStreams + if isinstance(ds, NetworkInputDStream)] + for stream in nis: + stream.startReceiver() + self.scheduler = Scheduler(self) self.scheduler.start(t or time.time()) + def _runOnce(self): + return self.scheduler.runOnce() + + def awaitTermination(self, timeout=None): + if self.scheduler is None: + raise RuntimeError('StreamimgContext not started') + + try: + deadline = time.time() + timeout if timeout is not None else None + while True: + is_terminated = self._runOnce() + if is_terminated or ( + deadline is not None and time.time() > deadline): + break + if self.batchCallback: + self.batchCallback() + except KeyboardInterrupt: + pass + finally: + # self.sc.stop() + logger.info("StreamingContext stopped successfully") + def stop(self): if self.scheduler: self.scheduler.stop() - if self.networkInputTracker: - self.networkInputTracker.stop() - if self.receiverJobThread: - self.receiverJobThread.stop() - self.sc.stop() - logger.info("StreamingContext stopped successfully") - def getSparkCheckpointDir(self, dir): - return os.path.join(dir, str(random.randint(0, 1000))) + def doCheckpoint(self, time): + if self.checkpointDuration and time >= self.lastCheckpointTime + \ + self.checkpointDuration: + self.lastCheckpointTime = time + self.graph.updateCheckpointData(time) + Checkpoint(self, time).write(self.checkpointDir) class Checkpoint(object): + def __init__(self, ssc, time): - self.ssc = ssc self.time = time self.master = ssc.sc.master - self.framework = ssc.sc.jobName self.graph = ssc.graph - self.checkpointDir = ssc.checkpointDir self.checkpointDuration = ssc.checkpointDuration + self.batchDuration = ssc.batchDuration - def write(self, dir): - pass + def write(self, path): + output_file = os.path.join(path, 'metadata') + with atomic_file(output_file) as f: + f.write(pickle.dumps(self, -1)) @classmethod - def read(cls, dir): - pass + def read(cls, path): + filename = os.path.join(path, 'metadata') + with open(filename, 'rb') as f: + return pickle.loads(f.read()) class Job(object): + def __init__(self, time, func): self.time = time self.func = func @@ -219,7 +306,9 @@ def run(self): def __str__(self): return '' % (self.func, self.time) + class JobManager(object): + def __init__(self, numThreads): pass @@ -229,7 +318,9 @@ def runJob(self, job): delayed = time.time() - job.time logger.info("job used %s, delayed %s", used, delayed) + class RecurringTimer(object): + def __init__(self, period, callback): self.period = period self.callback = callback @@ -237,7 +328,7 @@ def __init__(self, period, callback): self.stopped = False def start(self, start): - self.nextTime = (int(start / self.period) + 1) * self.period + self.nextTime = (int(start // self.period) + 1) * self.period self.stopped = False self.thread = spawn(self.run) logger.debug("RecurringTimer started, nextTime is %d", self.nextTime) @@ -252,19 +343,24 @@ def run(self): now = time.time() if now >= self.nextTime: logger.debug("start call %s with %d (delayed %f)", self.callback, - self.nextTime, now - self.nextTime) + self.nextTime, now - self.nextTime) self.callback(self.nextTime) self.nextTime += self.period else: time.sleep(max(min(self.nextTime - now, 1), 0.01)) + +_STOP, _EVENT = list(range(2)) + + class Scheduler(object): + def __init__(self, ssc): self.ssc = ssc self.graph = ssc.graph self.jobManager = JobManager(1) - #self.checkpointWriter = CheckpointWriter(ssc.checkpointDir) if ssc.checkpointDir else None - self.timer = RecurringTimer(ssc.batchDuration, self.generateRDDs) + self.timer = RecurringTimer(ssc.batchDuration, self.generateEvent) + self._queue = deque() def start(self, t): self.graph.start(t) @@ -274,22 +370,30 @@ def start(self, t): def stop(self): self.timer.stop() self.graph.stop() + self._queue.append((_STOP, None)) + + def generateEvent(self, time): + self._queue.append((_EVENT, time)) + + def runOnce(self): + try: + t, v = self._queue.popleft() + except IndexError: + time.sleep(0.1) + return False + + if t == _STOP: + return True + + self.generateRDDs(v) + return False def generateRDDs(self, time): for job in self.graph.generateRDDs(time): logger.debug("start to run job %s", job) self.jobManager.runJob(job) self.graph.forgetOldRDDs(time) - # self.doCheckpoint(time) - - #def doCheckpoint(self, time): - # return - # if self.ssc.checkpointDuration and (time-self.graph.zeroTime): - # startTime = time.time() - # self.ssc.graph.updateCheckpointData() - # Checkpoint(self.ssc, time).write(self.ssc.checkpointDir) - # stopTime = time.time() - # logger.info("Checkpointing the graph took %.0f ms", (stopTime - startTime)*1000) + self.ssc.doCheckpoint(time) class DStream(object): @@ -313,6 +417,7 @@ class DStream(object): * - A time interval at which the DStream generates an RDD * - A function that is used to generate an RDD after each time interval """ + def __init__(self, ssc): self.ssc = ssc self.slideDuration = None @@ -321,10 +426,25 @@ def __init__(self, ssc): self.generatedRDDs = {} self.rememberDuration = None self.mustCheckpoint = False - self.checkpointDuration = None - self.checkpointData = [] + self.last_checkpoint_time = 0 + self.checkpointData = {} self.graph = None + def __getstate__(self): + d = dict(self.__dict__) + d.pop('generatedRDDs') + d.pop('ssc') + return d + + def __setstate__(self, state): + self.__dict__.update(state) + self.generatedRDDs = {} + + def setContext(self, context): + self.ssc = context + for dep in self.dependencies: + dep.setContext(context) + @property def zeroTime(self): return self.graph.zeroTime @@ -333,35 +453,21 @@ def zeroTime(self): def parentRememberDuration(self): return self.rememberDuration - def checkpoint(self, interval): - self.checkpointDuration = interval - -# def initialize(self): -# if self.mustCheckpoint and not self.checkpointDuration: -# self.checkpointDuration = max(10, self.slideDuration) -# for dep in self.dependencies: -# dep.initialize() - - #def setContext(self, ssc): - # self.ssc = ssc - # for dep in self.dependencies: - # dep.setContext(ssc) - def setGraph(self, g): self.graph = g for dep in self.dependencies: dep.setGraph(g) def remember(self, duration): - if duration and duration > self.rememberDuration: + if duration and (self.rememberDuration is None or duration > self.rememberDuration): self.rememberDuration = duration for dep in self.dependencies: dep.remember(self.parentRememberDuration) def isTimeValid(self, t): d = (t - self.zeroTime) - dd = d / self.slideDuration * self.slideDuration - return abs(d-dd) < 1e-3 + dd = d // self.slideDuration * self.slideDuration + return abs(d - dd) < 1e-3 def compute(self, time): raise NotImplementedError @@ -372,49 +478,58 @@ def getOrCompute(self, time): if self.isTimeValid(time): rdd = self.compute(time) self.generatedRDDs[time] = rdd - # do checkpoint TODO + if (self.ssc.checkpointDuration and + time >= self.last_checkpoint_time + self.ssc.checkpointDuration): + self.last_checkpoint_time = time + if rdd: + rdd.checkpoint(self.ssc.checkpointDir) return rdd - #else: - #print 'invalid time', time, (time - self.zeroTime) / self.slideDuration * self.slideDuration def generateJob(self, time): rdd = self.getOrCompute(time) if rdd: - return Job(time, lambda : self.ssc.sc.runJob(rdd, lambda x:{})) + return Job(time, lambda: self.ssc.sc.runJob(rdd, lambda x: {})) def forgetOldRDDs(self, time): oldest = time - (self.rememberDuration or 0) - for k in self.generatedRDDs.keys(): + for k in list(self.generatedRDDs.keys()): if k < oldest: self.generatedRDDs.pop(k) for dep in self.dependencies: dep.forgetOldRDDs(time) def updateCheckpointData(self, time): - newRdds = [(t, rdd.getCheckpointFile) for t, rdd in self.generatedRDDs.items() - if rdd.getCheckpointFile] - oldRdds = self.checkpointData + newRdds = [(t, rdd.checkpoint_path) for t, rdd in self.generatedRDDs.items() + if rdd and rdd.checkpoint_path] + if newRdds: + oldRdds = self.checkpointData self.checkpointData = dict(newRdds) + for t, p in six.iteritems(oldRdds): + if t not in self.checkpointData: + try: + shutil.rmtree(p) + except OSError: + pass + for dep in self.dependencies: dep.updateCheckpointData(time) - ns = dict(newRdds) - for t, p in oldRdds: - if t not in ns: - os.unlink(p) - logger.info("remove %s %s", t, p) - logger.info("updated checkpoint data for time %s (%d)", time, len(newRdds)) + + logger.info( + "updated checkpoint data for time %s (%d)", + time, + len(newRdds)) def restoreCheckpointData(self): - for t, path in self.checkpointData: - self.generatedRDDs[t] = self.ssc.sc.checkpointFile(path) + for t, path in six.iteritems(self.checkpointData): + self.generatedRDDs[t] = CheckpointRDD(self.ssc.sc, path) for dep in self.dependencies: dep.restoreCheckpointData() logger.info("restoreCheckpointData") def slice(self, beginTime, endTime): rdds = [] - t = endTime # - self.slideDuration + t = endTime # - self.slideDuration while t > self.zeroTime and t > beginTime: rdd = self.getOrCompute(t) if rdd: @@ -445,10 +560,11 @@ def mapPartitions(self, func, preserve=False): return MapPartitionedDStream(self, func, preserve) def reduce(self, func): - return self.map(lambda x:(None, x)).reduceByKey(func, 1).map(lambda (x,y):y) + return self.map(lambda x: (None, x)).reduceByKey( + func, 1).map(lambda x_y1: x_y1[1]) def count(self): - return self.map(lambda x:1).reduce(lambda x,y:x+y) + return self.map(lambda x: 1).reduce(lambda x, y: x + y) def foreach(self, func): out = ForEachDStream(self, func) @@ -461,14 +577,15 @@ def transform(self, func): def show(self): def forFunc(rdd, t): some = rdd.take(11) - print "-" * 80 - print "Time:", time.asctime(time.localtime(t)) - print "-" * 80 + print("-" * 80) + print("Time:", time.asctime(time.localtime(t))) + print("-" * 80) for i in some[:10]: - print i + print(i) if len(some) > 10: - print '...' - print + print('...') + print() + return self.foreach(forFunc) def window(self, duration, slideDuration=None): @@ -481,13 +598,13 @@ def tumble(self, batch): def reduceByWindow(self, func, window, slideDuration=None, invFunc=None): if invFunc is not None: - return self.map(lambda x:(1, x)).reduceByKeyAndWindow(func, invFunc, - window, slideDuration, 1).map(lambda (x,y): y) + return self.map(lambda x: (1, x)).reduceByKeyAndWindow( + func, invFunc, window, slideDuration, 1).map(lambda x_y: x_y[1]) return self.window(window, slideDuration).reduce(func) def countByWindow(self, windowDuration, slideDuration=None): - return self.map(lambda x:1).reduceByWindow(lambda x,y:x+y, - windowDuration, slideDuration, lambda x,y:x-y) + return self.map(lambda x: 1).reduceByWindow( + lambda x, y: x + y, windowDuration, slideDuration, lambda x, y: x - y) def defaultPartitioner(self, part=None): if part is None: @@ -496,48 +613,58 @@ def defaultPartitioner(self, part=None): def groupByKey(self, numPart=None): createCombiner = lambda x: [x] - mergeValue = lambda l,x: l.append(x) or l - mergeCombiner = lambda x,y: x.extend(y) or x + mergeValue = lambda l, x: l.append(x) or l + mergeCombiner = lambda x, y: x.extend(y) or x if not isinstance(numPart, Partitioner): numPart = self.defaultPartitioner(numPart) - return self.combineByKey(createCombiner, mergeValue, mergeCombiner, numPart) + return self.combineByKey( + createCombiner, mergeValue, mergeCombiner, numPart) def reduceByKey(self, func, part=None): if not isinstance(part, Partitioner): part = self.defaultPartitioner(part) - return self.combineByKey(lambda x:x, func, func, part) + return self.combineByKey(lambda x: x, func, func, part) - def combineByKey(self, createCombiner, mergeValue, mergeCombiner, partitioner): + def combineByKey(self, createCombiner, mergeValue, + mergeCombiner, partitioner): agg = Aggregator(createCombiner, mergeValue, mergeCombiner) return ShuffledDStream(self, agg, partitioner) def countByKey(self, numPartitions=None): - return self.map(lambda (k,_):(k, 1)).reduceByKey(lambda x,y:x+y, numPartitions) + return self.map(lambda k__: (k__[0], 1)).reduceByKey( + lambda x, y: x + y, numPartitions) - def groupByKeyAndWindow(self, window, slideDuration=None, numPartitions=None): + def groupByKeyAndWindow( + self, window, slideDuration=None, numPartitions=None): return self.window(window, slideDuration).groupByKey(numPartitions) - def reduceByKeyAndWindow(self, func, invFunc, windowDuration, slideDuration=None, partitioner=None): + def reduceByKeyAndWindow( + self, func, invFunc, windowDuration, slideDuration=None, partitioner=None): if invFunc is None: - return self.window(windowDuration, slideDuration).reduceByKey(func, partitioner) + return self.window(windowDuration, slideDuration).reduceByKey( + func, partitioner) if slideDuration is None: slideDuration = self.slideDuration if not isinstance(partitioner, Partitioner): partitioner = self.defaultPartitioner(partitioner) - return ReducedWindowedDStream(self, func, invFunc, windowDuration, slideDuration, partitioner) + return ReducedWindowedDStream( + self, func, invFunc, windowDuration, slideDuration, partitioner) - def countByKeyAndWindow(self, windowDuration, slideDuration=None, numPartitions=None): - return self.map(lambda (k,_):(k, 1)).reduceByKeyAndWindow( - lambda x,y:x+y, lambda x,y:x-y, windowDuration, slideDuration, numPartitions) + def countByKeyAndWindow(self, windowDuration, + slideDuration=None, numPartitions=None): + return self.map(lambda k__2: (k__2[0], 1)).reduceByKeyAndWindow( + lambda x, y: x + y, lambda x, y: x - y, windowDuration, slideDuration, numPartitions) def updateStateByKey(self, func, partitioner=None, remember=True): if not isinstance(partitioner, Partitioner): partitioner = self.defaultPartitioner(partitioner) + def newF(it): for k, (vs, r) in it: nr = func(vs, r) if nr is not None: yield (k, nr) + return StateDStream(self, newF, partitioner, remember) def mapValues(self, func): @@ -552,12 +679,13 @@ def cogroup(self, other, partitioner=None): return CoGroupedDStream([self, other], partitioner) def join(self, other, partitioner=None): - return self.cogroup(other, partitioner).flatMapValues(lambda (x,y): itertools.product(x,y)) - + return self.cogroup(other, partitioner).flatMapValues( + lambda x_y3: itertools.product(x_y3[0], x_y3[1])) class DerivedDStream(DStream): transformer = None + def __init__(self, parent, func=None): DStream.__init__(self, parent.ssc) self.parent = parent @@ -565,34 +693,53 @@ def __init__(self, parent, func=None): self.dependencies = [parent] self.slideDuration = parent.slideDuration + def __getstate__(self): + d = DStream.__getstate__(self) + del d['func'] + d['_func'] = dump_func(self.func) + return d + + def __setstate__(self, state): + self.func = load_func(state.pop('_func')) + DStream.__setstate__(self, state) + def compute(self, t): rdd = self.parent.getOrCompute(t) if rdd: return getattr(rdd, self.transformer)(self.func) + class MappedDStream(DerivedDStream): transformer = 'map' + class FlatMappedDStream(DerivedDStream): transformer = 'flatMap' + class FilteredDStream(DerivedDStream): transformer = 'filter' + class MapValuedDStream(DerivedDStream): transformer = 'mapValue' + class FlatMapValuedDStream(DerivedDStream): transformer = 'flatMapValue' + class GlommedDStream(DerivedDStream): transformer = 'glom' + def compute(self, t): rdd = self.parent.getOrCompute(t) if rdd: return rdd.glom() + class MapPartitionedDStream(DerivedDStream): + def __init__(self, parent, func, preserve=True): DerivedDStream.__init__(self, parent, func) self.preserve = preserve @@ -600,34 +747,41 @@ def __init__(self, parent, func, preserve=True): def compute(self, t): rdd = self.parent.getOrCompute(t) if rdd: - return rdd.mapPartitions(self.func) # TODO preserve + return rdd.mapPartitions(self.func) # TODO preserve + class TransformedDStream(DerivedDStream): + def compute(self, t): rdd = self.parent.getOrCompute(t) if rdd: return self.func(rdd, t) + class ForEachDStream(DerivedDStream): + def compute(self, t): return self.parent.getOrCompute(t) def generateJob(self, time): rdd = self.getOrCompute(time) if rdd: - return Job(time, lambda :self.func(rdd, time)) + return Job(time, lambda: self.func(rdd, time)) + class StateDStream(DerivedDStream): - def __init__(self, parent, updateFunc, partitioner, preservePartitioning=True): + + def __init__(self, parent, updateFunc, partitioner, + preservePartitioning=True): DerivedDStream.__init__(self, parent, updateFunc) self.partitioner = partitioner self.preservePartitioning = preservePartitioning self.mustCheckpoint = True - self.rememberDuration = self.slideDuration # FIXME + self.rememberDuration = self.slideDuration # FIXME def compute(self, t): if t <= self.zeroTime: - #print 'less', t, self.zeroTime + # print 'less', t, self.zeroTime return prevRDD = self.getOrCompute(t - self.slideDuration) parentRDD = self.parent.getOrCompute(t) @@ -635,15 +789,21 @@ def compute(self, t): if prevRDD: if parentRDD: cogroupedRDD = parentRDD.cogroup(prevRDD) - return cogroupedRDD.mapValue(lambda (vs, rs):(vs, rs and rs[0] or None)).mapPartitions(updateFuncLocal) # preserve TODO + return cogroupedRDD.mapValue( + lambda vs_rs: (vs_rs[0], vs_rs[1] and vs_rs[1][0] or None)) \ + .mapPartitions(updateFuncLocal) # preserve TODO else: - return prevRDD + return prevRDD.mapValue( + lambda rs: ([], rs)).mapPartitions(updateFuncLocal) else: if parentRDD: groupedRDD = parentRDD.groupByKey(self.partitioner) - return groupedRDD.mapValue(lambda v:(v, None)).mapPartitions(updateFuncLocal) + return groupedRDD.mapValue( + lambda v: (v, None)).mapPartitions(updateFuncLocal) + class UnionDStream(DStream): + def __init__(self, parents): DStream.__init__(self, parents[0].ssc) self.parents = parents @@ -651,11 +811,13 @@ def __init__(self, parents): self.slideDuration = parents[0].slideDuration def compute(self, t): - rdds = filter(None, [p.getOrCompute(t) for p in self.parents]) + rdds = [_f for _f in [p.getOrCompute(t) for p in self.parents] if _f] if rdds: return self.ssc.sc.union(rdds) + class WindowedDStream(DStream): + def __init__(self, parent, windowDuration, slideDuration): DStream.__init__(self, parent.ssc) self.parent = parent @@ -676,21 +838,24 @@ def compute(self, t): class CoGroupedDStream(DStream): + def __init__(self, parents, partitioner): DStream.__init__(self, parents[0].ssc) self.parents = parents self.partitioner = partitioner - assert len(set([p.slideDuration for p in parents])) == 1, "the slideDuration must be same" + assert len(set([p.slideDuration for p in parents])) == 1, \ + "the slideDuration must be same" self.dependencies = parents self.slideDuration = parents[0].slideDuration def compute(self, t): - rdds = filter(None, [p.getOrCompute(t) for p in self.parents]) + rdds = [_f for _f in [p.getOrCompute(t) for p in self.parents] if _f] if rdds: return CoGroupedRDD(rdds, self.partitioner) class ShuffledDStream(DerivedDStream): + def __init__(self, parent, agg, partitioner): assert isinstance(parent, DStream) DerivedDStream.__init__(self, parent, agg) @@ -702,9 +867,11 @@ def compute(self, t): if rdd: return rdd.combineByKey(self.agg, self.partitioner) + class ReducedWindowedDStream(DerivedDStream): + def __init__(self, parent, func, invReduceFunc, - windowDuration, slideDuration, partitioner): + windowDuration, slideDuration, partitioner): DerivedDStream.__init__(self, parent, func) self.invfunc = invReduceFunc self.windowDuration = windowDuration @@ -720,6 +887,16 @@ def parentRememberDuration(self): return self.windowDuration + self.rememberDuration # persist + def __getstate__(self): + d = DerivedDStream.__getstate__(self) + del d['invfunc'] + d['_invfunc'] = dump_func(self.func) + return d + + def __setstate__(self, state): + self.invfunc = load_func(state.pop('_invfunc')) + DerivedDStream.__setstate__(self, state) + def compute(self, t): if t <= self.zeroTime: return @@ -730,17 +907,20 @@ def compute(self, t): oldRDDs = self.reducedStream.slice(prevWindow.begin, currWindow.begin) newRDDs = self.reducedStream.slice(prevWindow.end, currWindow.end) - prevWindowRDD = self.getOrCompute(prevWindow.end) or self.ssc.sc.makeRDD([]) + prevWindowRDD = (self.getOrCompute(prevWindow.end) + or self.ssc.sc.makeRDD([])) allRDDs = [prevWindowRDD] + oldRDDs + newRDDs cogroupedRDD = CoGroupedRDD(allRDDs, self.partitioner) nOld = len(oldRDDs) nNew = len(newRDDs) + def mergeValues(values): - #print values, nOld, nNew - assert len(values) == 1+nOld+nNew - oldValues = [values[i][0] for i in range(1, nOld+1) if values[i]] - newValues = [values[i][0] for i in range(1+nOld, nOld+1+nNew) if values[i]] + # print values, nOld, nNew + assert len(values) == 1 + nOld + nNew + oldValues = [values[i][0] for i in range(1, nOld + 1) if values[i]] + newValues = [values[i][0] + for i in range(1 + nOld, nOld + 1 + nNew) if values[i]] if not values[0]: if newValues: return reduce(reduceF, newValues) @@ -751,10 +931,12 @@ def mergeValues(values): if newValues: tmp = reduceF(tmp, reduce(reduceF, newValues)) return tmp + return cogroupedRDD.mapValue(mergeValues) class InputDStream(DStream): + def __init__(self, ssc): DStream.__init__(self, ssc) self.dependencies = [] @@ -766,23 +948,30 @@ def start(self): def stop(self): pass + class ConstantInputDStream(InputDStream): + def __init__(self, ssc, rdd): InputDStream.__init__(self, ssc) self.rdd = rdd + def compute(self, validTime): return self.rdd + def defaultFilter(path): if '/.' in path: return False return True + class ModTimeAndRangeFilter(object): - def __init__(self, lastModTime, filter): + + def __init__(self, lastModTime, filter, oldThreshold): self.lastModTime = lastModTime - self.filter = filter self.latestModTime = 0 + self.filter = filter + self.oldThreshold = oldThreshold self.accessedFiles = {} self.oldFiles = set() @@ -792,17 +981,23 @@ def __call__(self, path): if path in self.oldFiles: return - mtime = os.path.getmtime(path) - if mtime < self.lastModTime: - self.oldFiles.add(path) - return - if mtime > self.latestModTime: - self.latestModTime = mtime if os.path.islink(path): self.oldFiles.add(path) return + try: + mtime = os.path.getmtime(path) + except Exception as e: + logger.warning(str(e)) + return + if mtime < self.lastModTime: + if mtime < time.time() - self.oldThreshold: + self.oldFiles.add(path) + return + if mtime > self.latestModTime: + self.latestModTime = mtime + nsize = os.path.getsize(path) osize = self.accessedFiles.get(path, 0) if nsize <= osize: @@ -814,14 +1009,19 @@ def __call__(self, path): def rotate(self): self.lastModTime = self.latestModTime + class FileInputDStream(InputDStream): - def __init__(self, ssc, directory, filter=None, newFilesOnly=True): + + def __init__(self, ssc, directory, filter=None, newFilesOnly=True, oldThreshold=3600): InputDStream.__init__(self, ssc) - assert os.path.exists(directory), 'directory %s must exists' % directory + assert os.path.exists(directory), \ + 'directory %s must exists' % directory assert os.path.isdir(directory), '%s is not directory' % directory self.directory = directory - lastModTime = time.time() if newFilesOnly else 0 - self.filter = ModTimeAndRangeFilter(lastModTime, filter or defaultFilter) + lastModTime = time.time() - oldThreshold if newFilesOnly else 0 + filter = filter or defaultFilter + self.filter = ModTimeAndRangeFilter(lastModTime, filter, oldThreshold) + self.newFilesOnly = newFilesOnly def compute(self, validTime): files = [] @@ -833,10 +1033,75 @@ def compute(self, validTime): if files: self.filter.rotate() return self.ssc.sc.union([self.ssc.sc.partialTextFile(path, begin, end) - for path, begin, end in files]) + for path, begin, end in files]) + + def start(self): + if self.newFilesOnly: + # Init the new files old sizes. + for root, dirs, names in os.walk(self.directory): + for name in names: + self.filter(os.path.join(root, name)) + + +class RotatingFilesInputDStream(InputDStream): + def __init__(self, ssc, files): + InputDStream.__init__(self, ssc) + self.files = files + self._state = {} + + def start(self): + self._state = dict(self._get_state()) + + def _get_state(self): + for fn in self.files: + try: + realname = os.path.realpath(fn) + with closing(open_file(realname)) as f: + if f: + yield realname, (f.info.inode, f.info.length, f.info.mtime) + else: + st = os.stat(realname) + yield realname, (st.st_ino, st.st_size, st.st_mtime) + + except (OSError, Error): + pass + + def compute(self, validTime): + state = {} + offsets = {} + + for fn, (inode, size, mtime) in self._get_state(): + if fn not in self._state: + offsets[fn] = (0, size) + else: + _inode, _size, _mtime = self._state[fn] + if inode != _inode or (mtime > _mtime and size < _size): + offsets[fn] = (0, size) + elif mtime >= _mtime and size > _size: + offsets[fn] = (_size, size) + + state[fn] = (inode, size, mtime) + + for fn, (_inode, _size, _mtime) in six.iteritems(self._state): + if fn not in state: + try: + st = os.stat(fn) + inode, size, mtime = st.st_ino, st.st_size, st.st_mtime + except OSError: + continue + + if inode != _inode or (mtime > _mtime and size < _size): + offsets[fn] = (0, size) + elif mtime >= _mtime and size > _size: + offsets[fn] = (_size, size) + + self._state = state + return self.ssc.sc.union([self.ssc.sc.partialTextFile(path, begin, end) + for path, (begin, end) in six.iteritems(offsets)]) class QueueInputDStream(InputDStream): + def __init__(self, ssc, queue, oneAtAtime=True, defaultRDD=None): InputDStream.__init__(self, ssc) self.queue = queue @@ -854,42 +1119,151 @@ def compute(self, t): elif self.defaultRDD: return self.defaultRDD -class NetworkReceiverMessage: pass -class StopReceiver(NetworkReceiverMessage): - def __init__(self, msg): - self.msg = msg -class ReportBlock(NetworkReceiverMessage): - def __init__(self, blockId, metadata): - self.blockId = blockId - self.metadata = metadata -class ReportError(NetworkReceiverMessage): - def __init__(self, msg): - self.msg = msg - -class NetworkReceiver(object): - "TODO" class NetworkInputDStream(InputDStream): - def __init__(self, ssc): + + def __init__(self, ssc, func): InputDStream.__init__(self, ssc) - self.id = ssc.getNewNetworkStreamId() - def createReceiver(self): - return NetworkReceiver() - def compute(self, t): - blockIds = self.ssc.networkInputTracker.getBlockIds(self.id, t) - #return [BlockRDD(self.ssc.sc, blockIds)] + self.func = func + self._lock = threading.RLock() + self._messages = [] + + def __getstate__(self): + d = InputDStream.__getstate__(self) + del d['func'] + del d['_lock'] + d['_func'] = dump_func(self.func) + return d + + def __setstate__(self, state): + self.func = load_func(state.pop('_func')) + self._lock = threading.RLock() + InputDStream.__setstate__(self, state) + + def startReceiver(self): + def _run(): + while True: + generator = self.func() + try: + for message in generator: + if not self.ssc.sc.started: + return + with self._lock: + self._messages.append(message) + except: + logger.exception('fail to receive') + + spawn(_run) + def compute(self, t): + with self._lock: + if self._messages: + rdd = self.ssc.sc.makeRDD(self._messages) + self._messages = [] + return rdd -class SocketReceiver(NetworkReceiver): - pass class SocketInputDStream(NetworkInputDStream): - pass - - -class RawNetworkReceiver: - "TODO" -class RawInputDStream(NetworkInputDStream): - def createReceiver(self): - return RawNetworkReceiver() + def __init__(self, ssc, hostname, port): + NetworkInputDStream.__init__(self, ssc, self._receive) + self.hostname = hostname + self.port = port + + def __getstate__(self): + d = InputDStream.__getstate__(self) + del d['func'] + del d['_lock'] + return d + + def __setstate__(self, state): + self.func = self._receive + self._lock = threading.RLock() + InputDStream.__setstate__(self, state) + + def _receive(self): + client, f = None, None + try: + client = socket.socket() + client.connect((self.hostname, self.port)) + f = client.makefile() + for line in f: + yield line + except: + time.sleep(0.5 + random.random()) + raise + finally: + if f: + f.close() + if client: + client.close() + + +if WITH_SCRIBE: + class ScribeHandler(Iface): + def __init__(self, buf_que): + self.buf_que = buf_que + + def Log(self, messages): + try: + for m in messages: + self.buf_que.append(m.message) + except: + logger.exception('push message failed') + raise + return ResultCode.OK + + + class ScribeInputDStream(NetworkInputDStream): + def __init__(self, ssc, zk_address, zk_path, category='default'): + NetworkInputDStream.__init__(self, ssc, self._receive) + self.zk_address = zk_address + self.zk_path = zk_path + self.category = category + + def __getstate__(self): + d = InputDStream.__getstate__(self) + del d['func'] + del d['_lock'] + + def __setstate__(self, state): + self.func = self._receive + self._lock = threading.RLock() + InputDStream.__setstate__(self, state) + + def _createThriftServer(self): + buf_que = deque() + handler = ScribeHandler(buf_que) + protocol_factory = TBinaryProtocol.TBinaryProtocolFactory(False, False) + transport = TSocket.TServerSocket(host='0.0.0.0') + processor = Processor(handler) + server = TNonblockingServer.TNonblockingServer(processor, transport, + protocol_factory) + server._stop = False + while True: + try: + server.prepare() + port = transport.handle.getsockname()[1] + logger.info('get scribe port succeed: %d', port) + break + except socket.error: + pass + spawn(server.serve) + return server, port, buf_que + + def _receive(self): + server, port, buf_que = self._createThriftServer() + + kazoo_client = KazooClient(self.zk_address) + kazoo_client.start() + path = '%s/%s:%d' % (self.zk_path, socket.gethostname(), port) + kazoo_client.create(path, ephemeral=True, makepath=True) + + while not server._stop: + try: + message = buf_que.pop() + yield message + except: + time.sleep(0.1) + server.close() + kazoo_client.close() diff --git a/dpark/env.py b/dpark/env.py index 85ecb7d9..7f18cd71 100644 --- a/dpark/env.py +++ b/dpark/env.py @@ -1,99 +1,305 @@ -import os, logging -import time +from __future__ import absolute_import +import os import socket import shutil +import random +import signal +import uuid as uuid_pkg -import zmq - -from dpark import util +from dpark import utils +from dpark.utils.log import get_logger +from dpark.utils.memory import MemoryChecker import dpark.conf as conf -logger = logging.getLogger(__name__) +logger = get_logger(__name__) -class DparkEnv: - environ = {} - @classmethod - def register(cls, name, value): - cls.environ[name] = value - @classmethod - def get(cls, name, default=None): - return cls.environ.get(name, default) + +class TaskStats(object): def __init__(self): - self.started = False + self._reset() + + def _reset(self): + self.bytes_max_rss = 0 + self.secs_all = 0 + + # broadcast + self.secs_broadcast = 0 + + # shuffle: fetch and merge -> run and merge -> dump + self.bytes_fetch = 0 + self.bytes_dump = 0 + self.secs_fetch = 0 # 0 for sort merge if not use disk + self.secs_dump = 0 + + # rotate + self.num_fetch_rotate = 0 # 0 if all in memory + self.num_dump_rotate = 0 # 1 if all in memory + - def start(self, isMaster, environ={}): - if self.started: +def prepare_file_open(base, subpath): + path = os.path.join(base, subpath) + if os.path.exists(path): + os.remove(path) + else: + utils.mkdir_p(os.path.dirname(path)) + return path + + +class WorkDir(object): + + def __init__(self): + self.workdirs = [] + self.inited = False + self.seq_id = 0 + + @property + def next_id(self): + self.seq_id += 1 + return self.seq_id + + @property + def main(self): + return self.workdirs[0] + + def init(self, dpark_id): + if self.inited: return - logger.debug("start env in %s: %s %s", os.getpid(), - isMaster, environ) - self.isMaster = isMaster - if isMaster: - roots = conf.DPARK_WORK_DIR - if isinstance(roots, str): - roots = roots.split(',') - name = '%s-%s-%d' % (time.strftime("%Y%m%d-%H%M%S"), - socket.gethostname(), os.getpid()) - self.workdir = [os.path.join(root, name) for root in roots] - for d in self.workdir: + roots = conf.DPARK_WORK_DIR.split(",") + self.workdirs = [] + es = {} + for i, root in enumerate(roots): + try: + while not os.path.exists(root): + os.makedirs(root) + os.chmod(root, 0o777) # because umask + workdir = os.path.join(root, dpark_id) + self.workdirs.append(workdir) + except Exception as e: + es[root] = e + + if not self.workdirs: + raise Exception("workdirs not available: {}".format(es)) + + utils.mkdir_p(self.main) # executor will loc it + self.inited = True + + def export(self, tmppath, subpath): + if not os.path.exists(tmppath): + raise Exception("tmppath %s for % not exists", tmppath, subpath) + path = os.path.join(self.main, subpath) + if os.path.exists(path): + logger.warning("rm old localfile %s", path) + os.remove(path) + dirpath = os.path.dirname(path) + while not os.path.exists(dirpath): + utils.mkdir_p(dirpath) + logger.debug("export %s %s", tmppath, path) + os.symlink(tmppath, path) + return path + + def get_path(self, subpath): + return os.path.join(self.main, subpath) + + def _use_memdir(self, datasize): + if env.meminfo.rss + datasize > env.meminfo.mem_limit_soft: + return False + + st = os.statvfs(self.main) + free = st.f_bfree * st.f_bsize + ratio = st.f_bfree * 1.0 / st.f_blocks + if free < max(datasize, 1 << 30) or ratio < 0.66: + return False + + env.meminfo.add(datasize) + return True + + def _choose_disk_workdir(self): + disk_dirs = list(self.workdirs[1:]) + if not disk_dirs: + return self.workdirs[0] + + random.shuffle(disk_dirs) + for d in disk_dirs: + try: if not os.path.exists(d): - try: os.makedirs(d) - except OSError: pass - self.environ['SERVER_URI'] = 'file://' + self.workdir[0] - self.environ['WORKDIR'] = self.workdir - self.environ['COMPRESS'] = util.COMPRESS + utils.mkdir_p(d) + return d + except: + pass else: - self.environ.update(environ) - if self.environ['COMPRESS'] != util.COMPRESS: - raise Exception("no %s available" % self.environ['COMPRESS']) + logger.warning("_choose_disk_workdir fail") + return self.workdirs[0] - self.ctx = zmq.Context() + @classmethod + def _get_tmp_subpath(cls, prefix): + uuid = uuid_pkg.uuid4().hex # hex is short + subpath = '{}-{}-{}.temp'.format(prefix, uuid, os.getpid()) + return subpath + def alloc_tmp_dir(self, prefix, mem=False): + root = self.main if mem else self._choose_disk_workdir() + path = os.path.join(root, self._get_tmp_subpath(prefix)) + os.makedirs(path) + return path - from dpark.tracker import TrackerServer, TrackerClient - if isMaster: - self.trackerServer = TrackerServer() - self.trackerServer.start() - addr = self.trackerServer.addr - env.register('TrackerAddr', addr) - else: - addr = env.get('TrackerAddr') + def alloc_tmp_file(self, prefix, mem_first=False, datasize=0): + root = self.main if (mem_first and self._use_memdir(datasize)) else self._choose_disk_workdir() + path = os.path.join(root, self._get_tmp_subpath(prefix)) + return path - self.trackerClient = TrackerClient(addr) + def clean_up(self): + for d in self.workdirs: + while os.path.exists(d): + try: + shutil.rmtree(d, True) + except: + pass - from dpark.cache import CacheTracker - self.cacheTracker = CacheTracker() + def setup_cleaner_process(self): + ppid = os.getpid() + pid = os.fork() + if pid == 0: + os.setsid() + pid = os.fork() + if pid == 0: + try: + import psutil + except ImportError: + os._exit(1) - from dpark.shuffle import LocalFileShuffle, MapOutputTracker - LocalFileShuffle.initialize(isMaster) - self.mapOutputTracker = MapOutputTracker() - from dpark.shuffle import SimpleShuffleFetcher, ParallelShuffleFetcher - #self.shuffleFetcher = SimpleShuffleFetcher() - self.shuffleFetcher = ParallelShuffleFetcher(2) + try: + psutil.Process(ppid).wait() + os.killpg(ppid, signal.SIGKILL) # kill workers + except Exception: + pass # make sure to exit - from dpark.broadcast import start_manager - start_manager(isMaster) + finally: + self.clean_up() + os._exit(0) + os.wait() - self.started = True + +class DparkEnv(object): + + SERVER_URI = "SERVER_URI" + COMPRESS = "COMPRESS" + TRACKER_ADDR = "TRACKER_ADDR" + DPARK_ID = "DPARK_ID" + + def __init__(self): + self.environ = {} + self.task_stats = TaskStats() + self.workdir = WorkDir() + + self.master_started = False + self.slave_started = False + + # master + self.trackerServer = None + self.cacheTrackerServer = None + + # slave + # trackerServer clients + self.trackerClient = None + self.cacheTracker = None + # threads + self.meminfo = MemoryChecker() + self.shuffleFetcher = None + + def register(self, name, value): + self.environ[name] = value + + def get(self, name, default=None): + return self.environ.get(name, default) + + @property + def server_uri(self): + return self.environ[self.SERVER_URI] + + def start_master(self): + if self.master_started: + return + self.master_started = True + logger.debug("start master env in pid %s", os.getpid()) + + self._start_common() + self.register(self.COMPRESS, utils.COMPRESS) + + dpark_id = '{}-{}'.format(socket.gethostname(), uuid_pkg.uuid4()) + logger.info("dpark_id = %s", dpark_id) + self.register(self.DPARK_ID, dpark_id) + + self.workdir.init(dpark_id) + + self.register(self.SERVER_URI, 'file://' + self.workdir.main) # overwrited when executor starts + + # start centra servers in scheduler + from dpark.tracker import TrackerServer + self.trackerServer = TrackerServer() + self.trackerServer.start() + self.register(self.TRACKER_ADDR, self.trackerServer.addr) + + from dpark.cache import CacheTrackerServer + self.cacheTrackerServer = CacheTrackerServer() + + from dpark.broadcast import start_guide_manager + start_guide_manager() + + self._start_common() + + logger.debug("master env started") + + def _start_common(self): + if not self.shuffleFetcher: + from dpark.shuffle import ParallelShuffleFetcher + self.shuffleFetcher = ParallelShuffleFetcher(2) # start lazy, use also in scheduler(rdd.iterator) + + def start_slave(self): + """Called after env is updated.""" + if self.slave_started: + return + self.slave_started = True + + self._start_common() + + compress = self.get(self.COMPRESS) + if compress != utils.COMPRESS: + raise Exception("no %s available" % compress) + + # init clients + from dpark.tracker import TrackerClient + self.trackerClient = TrackerClient(self.get(self.TRACKER_ADDR)) + + from dpark.cache import CacheTracker + self.cacheTracker = CacheTracker() + + self.workdir.init(env.get(self.DPARK_ID)) logger.debug("env started") def stop(self): - if not getattr(self, 'started', False): - return - logger.debug("stop env in %s", os.getpid()) - self.shuffleFetcher.stop() - self.cacheTracker.stop() - self.mapOutputTracker.stop() - if self.isMaster: - self.trackerServer.stop() + logger.debug("stop env in pid %s", os.getpid()) + + if self.master_started: + if self.trackerServer is not None: + self.trackerServer.stop() + self.environ.pop(self.TRACKER_ADDR, None) + self.master_started = False + + if self.slave_started: + self.trackerClient.stop() + self.cacheTracker.stop() + self.slave_started = False + + if self.slave_started or self.master_started: + self.shuffleFetcher.stop() + from dpark.broadcast import stop_manager stop_manager() logger.debug("cleaning workdir ...") - for d in self.workdir: - shutil.rmtree(d, True) - logger.debug("done.") + self.workdir.clean_up() + logger.debug("env stopped.") - self.started = False env = DparkEnv() diff --git a/dpark/executor.py b/dpark/executor.py index 452de46a..28bbd231 100755 --- a/dpark/executor.py +++ b/dpark/executor.py @@ -1,42 +1,51 @@ -import logging -import os, sys, time +from __future__ import absolute_import +import gc +import os +import sys +import time +import errno +import fcntl import signal -import os.path +import socket +import logging import marshal -import cPickle -import multiprocessing import threading -import SocketServer -import SimpleHTTPServer -import shutil -import socket -import urllib2 -import platform -import gc -import time +import subprocess +import multiprocessing +import six +from six.moves import socketserver, cPickle, SimpleHTTPServer, urllib +import resource import zmq - -import pymesos as mesos -import pymesos.mesos_pb2 as mesos_pb2 +from addict import Dict +from pymesos import Executor, MesosExecutorDriver, encode_data, decode_data sys.path.insert(0, os.path.dirname(os.path.dirname(__file__))) -from dpark.util import compress, decompress, spawn +from dpark.utils import ( + compress, decompress, spawn, mkdir_p, DparkUserFatalError +) +from dpark.utils.log import get_logger, init_dpark_logger, formatter_message +from dpark.utils.memory import ERROR_TASK_OOM, set_oom_score from dpark.serialize import marshalable from dpark.accumulator import Accumulator -from dpark.schedule import Success, FetchFailed, OtherFailure from dpark.env import env -from dpark.shuffle import LocalFileShuffle from dpark.mutable_dict import MutableDict +from dpark.serialize import loads +from dpark.task import TTID, TaskState, TaskEndReason, FetchFailed +from dpark.utils.debug import spawn_rconsole +from dpark.shuffle import ShuffleWorkDir -logger = logging.getLogger("dpark.executor@%s" % socket.gethostname()) +logger = get_logger('dpark.executor') TASK_RESULT_LIMIT = 1024 * 256 DEFAULT_WEB_PORT = 5055 -MAX_WORKER_IDLE_TIME = 60 -MAX_EXECUTOR_IDLE_TIME = 60 * 60 * 24 +MAX_EXECUTOR_IDLE_TIME = 60 * 60 * 24 # 1 day +KILL_TIMEOUT = 0.1 # 0.1 sec, to reply to mesos fast +TASK_LOST_JOIN_TIMEOUT = 3 +TASK_LOST_DISCARD_TIMEOUT = 60 Script = '' + def setproctitle(x): try: from setproctitle import setproctitle as _setproctitle @@ -44,28 +53,34 @@ def setproctitle(x): except ImportError: pass -def reply_status(driver, task_id, state, data=None): - status = mesos_pb2.TaskStatus() - status.task_id.MergeFrom(task_id) + +def reply_status(driver, task_id, state, reason=None, msg=None, data=None): + status = Dict() + status.task_id = task_id status.state = state + if reason is not None: + status.message = '{}:{}'.format(reason, msg) status.timestamp = time.time() if data is not None: - status.data = data + status.data = encode_data(data) driver.sendStatusUpdate(status) + def run_task(task_data): try: gc.disable() - task, ntry = cPickle.loads(decompress(task_data)) + task, task_try_id = loads(decompress(task_data)) + ttid = TTID(task_try_id) Accumulator.clear() - result = task.run(ntry) + result = task.run(ttid.ttid) + env.task_stats.bytes_max_rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss * 1024 accUpdate = Accumulator.values() MutableDict.flush() if marshalable(result): try: flag, data = 0, marshal.dumps(result) - except Exception, e: + except Exception: flag, data = 1, cPickle.dumps(result, -1) else: @@ -73,37 +88,47 @@ def run_task(task_data): data = compress(data) if len(data) > TASK_RESULT_LIMIT: - path = LocalFileShuffle.getOutputFile(0, ntry, task.id, len(data)) - f = open(path, 'w') - f.write(data) - f.close() - data = '/'.join([LocalFileShuffle.getServerUri()] + path.split('/')[-3:]) + # shuffle_id start from 1 + swd = ShuffleWorkDir(0, task.id, ttid.task_try) + tmppath = swd.alloc_tmp(len(data)) + with open(tmppath, 'wb') as f: + f.write(data) + f.close() + path = swd.export(tmppath) + data = '/'.join( + [env.server_uri] + path.split('/')[-3:] + ) flag += 2 - return mesos_pb2.TASK_FINISHED, cPickle.dumps((Success(), (flag, data), accUpdate), -1) - except FetchFailed, e: - return mesos_pb2.TASK_FAILED, cPickle.dumps((e, None, None), -1) - except : + return TaskState.finished, cPickle.dumps(((flag, data), accUpdate, env.task_stats), -1) + except FetchFailed as e: + return TaskState.failed, TaskEndReason.fetch_failed, str(e), cPickle.dumps(e) + except Exception as e: import traceback msg = traceback.format_exc() - return mesos_pb2.TASK_FAILED, cPickle.dumps((OtherFailure(msg), None, None), -1) + ename = e.__class__.__name__ + fatal_exceptions = (DparkUserFatalError, ArithmeticError, + ValueError, LookupError, SyntaxError, + TypeError, AssertionError) + prefix = "FATAL" if isinstance(e, fatal_exceptions) else "FAILED" + return TaskState.failed, '{}_EXCEPTION_{}'.format(prefix, ename), msg, cPickle.dumps(e) finally: gc.collect() gc.enable() -def init_env(args): - env.start(False, args) - class LocalizedHTTP(SimpleHTTPServer.SimpleHTTPRequestHandler): basedir = None + def translate_path(self, path): - out = SimpleHTTPServer.SimpleHTTPRequestHandler.translate_path(self, path) + out = SimpleHTTPServer.SimpleHTTPRequestHandler.translate_path( + self, path) return self.basedir + '/' + os.path.relpath(out) def log_message(self, format, *args): pass + def startWebServer(path): # check the default web server if not os.path.exists(path): @@ -112,282 +137,402 @@ def startWebServer(path): with open(testpath, 'w') as f: f.write(path) default_uri = 'http://%s:%d/%s' % (socket.gethostname(), DEFAULT_WEB_PORT, - os.path.basename(path)) + os.path.basename(path)) try: - data = urllib2.urlopen(default_uri + '/' + 'test').read() - if data == path: + data = urllib.request.urlopen(default_uri + '/' + 'test').read() + if data == path.encode('utf-8'): return default_uri - except IOError, e: + except IOError: pass - logger.warning("default webserver at %s not available", DEFAULT_WEB_PORT) + logger.warning('default webserver at %s not available', DEFAULT_WEB_PORT) LocalizedHTTP.basedir = os.path.dirname(path) - ss = SocketServer.TCPServer(('0.0.0.0', 0), LocalizedHTTP) + ss = socketserver.TCPServer(('0.0.0.0', 0), LocalizedHTTP) spawn(ss.serve_forever) - uri = "http://%s:%d/%s" % (socket.gethostname(), ss.server_address[1], - os.path.basename(path)) + uri = 'http://%s:%d/%s' % (socket.gethostname(), ss.server_address[1], + os.path.basename(path)) return uri -def forward(fd, addr, prefix=''): - f = os.fdopen(fd, 'r') - ctx = zmq.Context() - out = [None] - buf = [] - def send(buf): - if not out[0]: - out[0] = ctx.socket(zmq.PUSH) - out[0].connect(addr) - out[0].send(prefix+''.join(buf)) - - while True: - try: - line = f.readline() - if not line: break - buf.append(line) - if line.endswith('\n'): - send(buf) - buf = [] - except IOError: - break - if buf: - send(buf) - if out[0]: - out[0].close() - f.close() - ctx.shutdown() - -def get_pool_memory(pool): + +def terminate(tid, proc): + name = 'worker(tid: %s, pid: %s)' % (tid, proc.pid) try: - import psutil - p = psutil.Process(pool._pool[0].pid) - return p.get_memory_info()[0] >> 20 - except Exception: - return 0 + os.kill(proc.pid, signal.SIGTERM) + except Exception as e: + if proc.join(timeout=KILL_TIMEOUT / 2) is None: + try: + os.kill(proc.pid, signal.SIGKILL) + except Exception as e: + if proc.join(KILL_TIMEOUT / 2) is not None: + logger.exception('%s terminate fail', name) + def get_task_memory(task): for r in task.resources: if r.name == 'mem': return r.scalar.value - logger.error("no memory in resource: %s", task.resources) - return 100 # 100M + logger.error('no memory in resource: %s', task.resources) + return 100 # 100M + def safe(f): def _(self, *a, **kw): with self.lock: r = f(self, *a, **kw) return r + return _ -def setup_cleaner_process(workdir): - ppid = os.getpid() - pid = os.fork() - if pid == 0: - os.setsid() - pid = os.fork() - if pid == 0: - try: - import psutil - except ImportError: - os._exit(1) + +class Redirect(object): + + def __init__(self, fd, addr, prefix): + self.fd = fd + self.addr = addr + self.prefix = prefix + + self.fd_dup = os.dup(self.fd) + self.origin_wfile = None + + self.pipe_rfd, self.pipe_wfd = os.pipe() + self.pipe_rfile = os.fdopen(self.pipe_rfd, 'rb') + self.pipe_wfile = os.fdopen(self.pipe_wfd, 'wb', 0) + + os.close(self.fd) + os.dup2(self.pipe_wfd, self.fd) + # assert os.dup(self.pipe_wfd) == self.fd, 'redirect io failed' + + self.ctx = zmq.Context() + self._shutdown = False + self.thread = None + self.sock = None + + self.thread = spawn(self._forward) + + def reset(self): + err = None + try: + self._shutdown = True + self.pipe_wfile.close() + os.close(self.fd) + + self.thread.join(1) + if self.sock: + self.sock.close() + self.ctx.destroy() + except Exception as e: + err = e + + os.dup2(self.fd_dup, self.fd) # will close fd first + self.origin_wfile = os.fdopen(self.fd, 'wb', 0) + + logger.debug('should see me in sandbox') + if err: + logger.error('redirect reset err:', err) + + if self.thread.isAlive(): + logger.error('redirect thread not exit') + + return self.origin_wfile + + def _send(self, buf): + if not self.sock: + self.sock = self.ctx.socket(zmq.PUSH) + self.sock.setsockopt(zmq.LINGER, 0) + self.sock.connect(self.addr) + + data = self.prefix + ''.join(buf) + + while not self._shutdown: try: - psutil.Process(ppid).wait() - os.killpg(ppid, signal.SIGKILL) # kill workers - except Exception, e: - pass # make sure to exit - finally: - for d in workdir: - while os.path.exists(d): - try: shutil.rmtree(d, True) - except: pass - os._exit(0) - os.wait() - -class MyExecutor(mesos.Executor): + self.sock.send(data, zmq.NOBLOCK) + return + except zmq.Again: + time.sleep(0.1) + continue + + def _forward(self): + buf = [] + try: + while not self._shutdown: + try: + line = self.pipe_rfile.readline() + if not line: + break + buf.append(line) + if line.endswith('\n'): + self._send(buf) + buf = [] + except IOError: + break + if buf: + self._send(buf) + except Exception as e: + logger.error('_forward err: %s', e) + + +class MyExecutor(Executor): + def __init__(self): - self.workdir = [] - self.idle_workers = [] - self.busy_workers = {} - self.lock = threading.RLock() + # task_id.value -> (task, process) + self.tasks = {} + + # task_id.value -> (task, timestamp) + self.finished_tasks = {} - self.stdout, wfd = os.pipe() - sys.stdout = os.fdopen(wfd, 'w', 0) - os.close(1) - assert os.dup(wfd) == 1, 'redirect io failed' + # (task_id.value, (status, data)) + self.result_queue = multiprocessing.Queue() - self.stderr, wfd = os.pipe() - sys.stderr = os.fdopen(wfd, 'w', 0) - os.close(2) - assert os.dup(wfd) == 2, 'redirect io failed' + self.lock = threading.RLock() + + # Keep the file descriptor of current workdir, + # so we can check whether a workdir is in use externally. + self._fd_for_locks = [] + self.stdout_redirect = None + self.stderr_redirect = None - def check_memory(self, driver): + def check_alive(self, driver): try: import psutil except ImportError: - logger.error("no psutil module") + logger.error('no psutil module') return - mem_limit = {} idle_since = time.time() + kill_ecs = [-signal.SIGTERM] + while True: - self.lock.acquire() + with self.lock: + tasks = self.tasks.items() - for tid, (task, pool) in self.busy_workers.items(): + tids_to_pop = [] + for tid, (task, proc) in tasks: task_id = task.task_id - try: - pid = pool._pool[0].pid - p = psutil.Process(pid) - rss = p.get_memory_info()[0] >> 20 - except Exception, e: - logger.error("worker process %d of task %s is dead: %s", pid, tid, e) - reply_status(driver, task_id, mesos_pb2.TASK_LOST) - self.busy_workers.pop(tid) - continue - - if p.status == psutil.STATUS_ZOMBIE or not p.is_running(): - logger.error("worker process %d of task %s is zombie", pid, tid) - reply_status(driver, task_id, mesos_pb2.TASK_LOST) - self.busy_workers.pop(tid) - continue - - offered = get_task_memory(task) - if not offered: - continue - if rss > offered * 1.5: - logger.warning("task %s used too much memory: %dMB > %dMB * 1.5, kill it. " - + "use -M argument or taskMemory to request more memory.", tid, rss, offered) - reply_status(driver, task_id, mesos_pb2.TASK_KILLED) - self.busy_workers.pop(tid) - pool.terminate() - elif rss > offered * mem_limit.get(tid, 1.0): - logger.debug("task %s used too much memory: %dMB > %dMB, " - + "use -M to request or taskMemory for more memory", tid, rss, offered) - mem_limit[tid] = rss / offered + 0.1 + name = "task %s (pid = %d)" % (tid, proc.pid) + proc_end = False + reason = None + msg = None + try: + p = psutil.Process(proc.pid) + except Exception: + proc_end = True + + if proc_end or p.status() == psutil.STATUS_ZOMBIE or (not p.is_running()): + proc.join(TASK_LOST_JOIN_TIMEOUT) # join in py2 not return exitcode + ec = proc.exitcode + if ec == 0: # p.status() == psutil.STATUS_ZOMBIE + continue # handled in replier + + if ec is not None: + proc_end = True + msg = 'exitcode: {}'.format(ec) + if ec in kill_ecs: + reason = TaskEndReason.recv_sig + elif ec == -signal.SIGKILL: + reason = TaskEndReason.recv_sig_kill + elif ec == ERROR_TASK_OOM: + reason = TaskEndReason.task_oom + else: + reason = TaskEndReason.other_ecs + logger.warning('%s lost with exit code: %s', tid, ec) + else: + try: + os.waitpid(proc.pid, os.WNOHANG) + except OSError as e: + proc_end = True + if e.errno != errno.ECHILD: + logger.exception('%s lost, raise exception when waitpid', tid) + else: + t = self.finished_tasks.get(tid) + if t is not None and time.time() - t > TASK_LOST_DISCARD_TIMEOUT: + logger.warning('%s is zombie for %d secs, discard it!', name, TASK_LOST_DISCARD_TIMEOUT) + + if proc_end: + tids_to_pop.append(tid) + reply_status(driver, task_id, TaskState.failed, reason, msg) + + with self.lock: + for tid_ in tids_to_pop: + try: + self.tasks.pop(tid_) + except: + pass now = time.time() - n = len([1 for t, p in self.idle_workers if t + MAX_WORKER_IDLE_TIME < now]) - if n: - for _, p in self.idle_workers[:n]: - p.terminate() - self.idle_workers = self.idle_workers[n:] - - if self.busy_workers or self.idle_workers: + if self.tasks: idle_since = now elif idle_since + MAX_EXECUTOR_IDLE_TIME < now: os._exit(0) - self.lock.release() - time.sleep(1) + def _try_flock(self, path): + fd = os.open(path, os.O_RDONLY) + try: + fcntl.flock(fd, fcntl.LOCK_EX | fcntl.LOCK_NB) + except IOError as e: + try: + pids = subprocess.check_output(['fuser', path]).split() + curr_pid = os.getpid() + logger.warning( + 'current process: %s, processes that are using %s: %s', + curr_pid, path, pids) + except Exception: + pass + raise e + + self._fd_for_locks.append(fd) + @safe - def registered(self, driver, executorInfo, frameworkInfo, slaveInfo): + def registered(self, driver, executorInfo, frameworkInfo, agent_info): try: global Script - Script, cwd, python_path, osenv, self.parallel, out_logger, err_logger, logLevel, args = marshal.loads(executorInfo.data) - self.init_args = args + ( + Script, cwd, python_path, osenv, self.parallel, + out_logger, err_logger, logLevel, use_color, dpark_env + ) = marshal.loads(decode_data(executorInfo.data)) + sys.path = python_path os.environ.update(osenv) - setproctitle(Script) + setproctitle('[Executor]' + Script) + + prefix = formatter_message( + '{MAGENTA}[%s]{RESET} ' % socket.gethostname().ljust(10), + use_color + ) + + init_dpark_logger(logLevel, use_color=use_color) + logging.root.setLevel(logLevel) - prefix = '[%s] ' % socket.gethostname() - self.outt = spawn(forward, self.stdout, out_logger, prefix) - self.errt = spawn(forward, self.stderr, err_logger, prefix) - logging.basicConfig(format='%(asctime)-15s [%(levelname)s] [%(name)-9s] %(message)s', level=logLevel) + r1 = self.stdout_redirect = Redirect(1, out_logger, prefix) + sys.stdout = r1.pipe_wfile + + r2 = self.stderr_redirect = Redirect(2, err_logger, prefix) + sys.stderr = r2.pipe_wfile + + spawn_rconsole(locals()) if os.path.exists(cwd): try: os.chdir(cwd) - except Exception, e: - logger.warning("change cwd to %s failed: %s", cwd, e) + except Exception as e: + logger.warning('change cwd to %s failed: %s', cwd, e) else: - logger.warning("cwd (%s) not exists", cwd) + logger.warning('cwd (%s) not exists', cwd) + + env.workdir.init(dpark_env.get(env.DPARK_ID)) + self._try_flock(env.workdir.main) + dpark_env['SERVER_URI'] = startWebServer(env.workdir.main) + if 'MESOS_SLAVE_PID' in os.environ: # make unit test happy + env.workdir.setup_cleaner_process() - self.workdir = args['WORKDIR'] - root = os.path.dirname(self.workdir[0]) - if not os.path.exists(root): - os.mkdir(root) - os.chmod(root, 0777) # because umask - args['SERVER_URI'] = startWebServer(self.workdir[0]) - if 'MESOS_SLAVE_PID' in os.environ: # make unit test happy - setup_cleaner_process(self.workdir) + spawn(self.check_alive, driver) + spawn(self.replier, driver) - spawn(self.check_memory, driver) + env.environ.update(dpark_env) + from dpark.broadcast import start_download_manager + start_download_manager() - logger.debug("executor started at %s", slaveInfo.hostname) + logger.debug('executor started at %s', agent_info.hostname) - except Exception, e: + except Exception as e: import traceback msg = traceback.format_exc() - logger.error("init executor failed: %s", msg) + logger.error('init executor failed: %s', msg) raise - def get_idle_worker(self): - try: - return self.idle_workers.pop()[1] - except IndexError: - p = multiprocessing.Pool(1, init_env, [self.init_args]) - p.done = 0 - return p + def replier(self, driver): + while True: + try: + result = self.result_queue.get() + if result is None: + return + + reason = None + message = None + + task_id_value, result = result + if result[0] == TaskState.failed: + state, reason, message, data = result + else: + state, data = result + + with self.lock: + task, _ = self.tasks.pop(task_id_value) + self.finished_tasks[task_id_value] = time.time() + + reply_status(driver, task.task_id, state, reason, message, data) + + except Exception as e: + logger.warning('reply fail %s', e) @safe def launchTask(self, driver, task): task_id = task.task_id - reply_status(driver, task_id, mesos_pb2.TASK_RUNNING) - logger.debug("launch task %s", task.task_id.value) - try: - def callback((state, data)): - reply_status(driver, task_id, state, data) - with self.lock: - _, pool = self.busy_workers.pop(task.task_id.value) - pool.done += 1 - self.idle_workers.append((time.time(), pool)) + reply_status(driver, task_id, TaskState.running) + logger.debug('launch task %s', task.task_id.value) - pool = self.get_idle_worker() - self.busy_workers[task.task_id.value] = (task, pool) - pool.apply_async(run_task, [task.data], callback=callback) + def worker(procname, q, task_id_value, task_data): + task_id_str = "task %s" % (task_id_value,) + threading.current_thread().name = task_id_str + setproctitle(procname) + set_oom_score(100) + env.start_slave() + q.put((task_id_value, run_task(task_data))) - except Exception, e: + try: + name = '[Task-%s]%s' % (task.task_id.value, Script) + proc = multiprocessing.Process(target=worker, + args=(name, + self.result_queue, + task.task_id.value, + decode_data(task.data),)) + proc.name = name + proc.daemon = True + proc.start() + self.tasks[task.task_id.value] = (task, proc) + + except Exception as e: import traceback msg = traceback.format_exc() - reply_status(driver, task_id, mesos_pb2.TASK_LOST, msg) + reply_status(driver, task_id, TaskState.failed, TaskEndReason.launch_failed, msg, cPickle.dumps(e)) @safe def killTask(self, driver, taskId): - reply_status(driver, taskId, mesos_pb2.TASK_KILLED) - if taskId.value in self.busy_workers: - task, pool = self.busy_workers.pop(taskId.value) - pool.terminate() + reply_status(driver, taskId, TaskState.killed) + if taskId.value in self.tasks: + _, proc = self.tasks.pop(taskId.value) + terminate(taskId.value, proc) @safe def shutdown(self, driver=None): - def terminate(p): - try: - for pi in p._pool: - os.kill(pi.pid, signal.SIGKILL) - except Exception, e: - pass - for _, p in self.idle_workers: - terminate(p) - for _, p in self.busy_workers.itervalues(): - terminate(p) - - # clean work files - for d in self.workdir: - try: shutil.rmtree(d, True) - except: pass - - sys.stdout.close() - sys.stderr.close() - os.close(1) - os.close(2) - self.outt.join() - self.errt.join() + for tid, (_, proc) in six.iteritems(self.tasks): + terminate(tid, proc) + self.tasks = {} + self.result_queue.put(None) + for fd in self._fd_for_locks: + os.close(fd) + if self.stdout_redirect: + sys.stdout = self.stdout_redirect.reset() + if self.stderr_redirect: + sys.stderr = self.stderr_redirect.reset() + def run(): + setproctitle('Executor') + if os.getuid() == 0: + gid = os.environ['GID'] + uid = os.environ['UID'] + os.setgid(int(gid)) + os.setuid(int(uid)) + executor = MyExecutor() - driver = mesos.MesosExecutorDriver(executor) + driver = MesosExecutorDriver(executor, use_addict=True) driver.run() + if __name__ == '__main__': + fmt = '%(asctime)-15s [%(levelname)s] [%(threadName)s] [%(name)-9s] %(message)s' + logging.basicConfig(format=fmt, level=logging.INFO) run() diff --git a/dpark/file_manager/__init__.py b/dpark/file_manager/__init__.py new file mode 100644 index 00000000..4156be1e --- /dev/null +++ b/dpark/file_manager/__init__.py @@ -0,0 +1,43 @@ +from __future__ import absolute_import +from .utils import FileInfo, unpack, read_chunk +from .consts import * +from .mfs_proxy import ProxyConn +from dpark.utils.log import get_logger +from .fs import MooseFS, PosixFS +import os + +logger = get_logger(__name__) + + +class FileManager(object): + def __init__(self): + self.fs_list = [MooseFS(), PosixFS()] + + def register_fs(self, fs): + self.fs_list = [fs] + self.fs_list + + def _get_fs(self, path): + for fs in self.fs_list: + if fs.check_ok(path): + return fs + return None + + def open_file(self, path): + path = os.path.realpath(path) + fs = self._get_fs(path) + return fs.open_file(path) + + def walk(self, path, followlinks=True): + fs = self._get_fs(path) + return fs.walk(path, followlinks=followlinks) + + +file_manager = FileManager() + + +def open_file(path): + return file_manager.open_file(path) + + +def walk(path, followlinks=True): + return file_manager.walk(path, followlinks=followlinks) diff --git a/dpark/file_manager/consts.py b/dpark/file_manager/consts.py new file mode 100644 index 00000000..87b70369 --- /dev/null +++ b/dpark/file_manager/consts.py @@ -0,0 +1,106 @@ +CHUNKSIZE = 1 << 26 + +GETDIR_FLAG_WITHATTR = 0x01 +GETDIR_FLAG_ADDTOCACHE = 0x02 + +# type for readdir command +TYPE_FILE = 'f' +TYPE_SYMLINK = 'l' +TYPE_DIRECTORY = 'd' +TYPE_FIFO = 'q' +TYPE_BLOCKDEV = 'b' +TYPE_CHARDEV = 'c' +TYPE_SOCKET = 's' +TYPE_TRASH = 't' +TYPE_SUSTAINED = 'r' +TYPE_UNKNOWN = '?' + +ERROR_MAX = 38 +ERROR_LOCKED = 11 + +# CHUNKSERVER <-> CLIENT/CHUNKSERVER +CLTOCS_READ = 200 +# chunkid:64 version:32 offset:32 size:32 +CSTOCL_READ_STATUS = 201 +# chunkid:64 status:8 +CSTOCL_READ_DATA = 202 +# chunkid:64 blocknum:16 offset:16 size:32 crc:32 size*[ databyte:8 ] + +CLTOMA_FUSE_GETATTR = 408 +# msgid:32 inode:32 +# msgid:32 inode:32 uid:32 gid:32 +MATOCL_FUSE_GETATTR = 409 +# msgid:32 status:8 +# msgid:32 attr:35B +CLTOMA_FUSE_READLINK = 412 +# msgid:32 inode:32 +MATOCL_FUSE_READLINK = 413 +# msgid:32 status:8 +# msgid:32 length:32 path:lengthB +CLTOMA_FUSE_READDIR = 428 +# msgid:32 inode:32 uid:32 gid:32 - old version (works like new version with flags==0 +# msgid:32 inode:32 uid:32 gid:32 flags:8 +MATOCL_FUSE_READDIR = 429 +# msgid:32 status:8 +# msgid:32 N*[ name:NAME inode:32 type:8 ] - when GETDIR_FLAG_WITHATTR in flags is not set +# msgid:32 N*[ name:NAME inode:32 type:35B ] - when GETDIR_FLAG_WITHATTR in flags is set + +CLTOMA_FUSE_READ_CHUNK = 432 +# msgid:32 inode:32 chunkindx:32 +MATOCL_FUSE_READ_CHUNK = 433 +# msgid:32 status:8 +# msgid:32 length:64 chunkid:64 version:32 N*[ip:32 port:16] +# msgid:32 length:64 srcs:8 srcs*[chunkid:64 version:32 ip:32 port:16] - not implemented + +errtab = [ + "OK", + "Operation not permitted", + "Not a directory", + "No such file or directory", + "Permission denied", + "File exists", + "Invalid argument", + "Directory not empty", + "Chunk lost", + "Out of memory", + "Index too big", + "Chunk locked", + "No chunk servers", + "No such chunk", + "Chunk is busy", + "Incorrect register BLOB", + "None of chunk servers performed requested operation", + "File not opened", + "Write not started", + "Wrong chunk version", + "Chunk already exists", + "No space left", + "IO error", + "Incorrect block number", + "Incorrect size", + "Incorrect offset", + "Can't connect", + "Incorrect chunk id", + "Disconnected", + "CRC error", + "Operation delayed", + "Can't create path", + "Data mismatch", + "Read-only file system", + "Quota exceeded", + "Bad session id", + "Password is needed", + "Incorrect password", + "Unknown MFS error", +] + + +def strerror(code): + if code > ERROR_MAX: + code = ERROR_MAX + return errtab[code] + + +S_IFDIR = 0o040000 # directory */ +S_IFREG = 0o100000 # regular */ +S_IFLNK = 0o120000 # symbolic link */ diff --git a/dpark/file_manager/fs.py b/dpark/file_manager/fs.py new file mode 100644 index 00000000..ae824023 --- /dev/null +++ b/dpark/file_manager/fs.py @@ -0,0 +1,392 @@ +from __future__ import absolute_import +import os +import stat +import errno +import socket +import threading +from .utils import FileInfo, read_chunk +from .consts import * +from .mfs_proxy import ProxyConn +from dpark.utils.log import get_logger +import six +from six.moves import range + +try: + from cStringIO import StringIO +except ImportError: + from six import BytesIO as StringIO + +logger = get_logger(__name__) + + +class FileSystem(object): + + def readlink(self, path): + raise NotImplementedError + + def open_file(self, path): + raise NotImplementedError + + def listdir(self, path): + raise NotImplementedError + + def walk(self, path, followlinks=True): + raise NotImplementedError + + def check_ok(self, path): + raise NotImplementedError + + +class PosixFS(object): + + def readlink(self, path): + return os.readlink(path) + + def open_file(self, path): + return PosixFile(path) + + def listdir(self, path): + return os.listdir(path) + + def walk(self, path, followlinks): + return os.walk(path, followlinks=followlinks) + + def check_ok(self, path): + return True + + +class MooseFS(PosixFS): + + def __init__(self): + self._local = threading.local() + + def _find_proxy(self, path): + pid = getattr(self._local, 'pid', None) + if pid != os.getpid(): + self._local.proxy_map = {} + self._local.pid = os.getpid() + self.proxy_map = self._local.proxy_map + + for mountpoint in self.proxy_map: + if mountpoint in path: + return self.proxy_map[mountpoint] + + dir_path = path if os.path.isdir(path) else os.path.dirname(path) + mount = '' + while os.path.exists(os.path.join(dir_path, '.masterinfo')): + mount = dir_path + dir_path = os.path.dirname(dir_path) + + if not mount: + raise OSError('Can not find mount for %s' % path) + + host, port, version = ProxyConn.get_masterinfo(os.path.join(mount, '.masterinfo')) + self.proxy_map[mount] = ProxyConn(host, port, version) + return self.proxy_map[mount] + + def _get_indeed_root(self, root): + root = os.path.realpath(root) + proxy = self._find_proxy(root) + try: + st = os.lstat(root) + return root, proxy, st.st_ino + except OSError as e: + if e.errno == errno.ENOENT: + return None, proxy, None + raise e + + def walk(self, path, followlinks=False): + ds = [path] + while ds: + root = ds.pop() + real_root, proxy, inode = self._get_indeed_root(root) + if not real_root: + logger.warning('path not exists: %s', root) + continue + if not proxy and followlinks: + logger.warning('the path to walk is symlink to local: %s', root) + for sub_root, dirs, names in os.walk(real_root, followlinks=followlinks): + rel_path = os.path.relpath(sub_root, real_root) + rel_dir = rel_path if not rel_path.startswith('.') else '' + yield os.path.join(root, rel_dir), dirs, names + continue + cs = proxy.getdirplus(inode) + dirs, files = [], [] + for name, info in six.iteritems(cs): + if name in '..': + continue + if info.ftype == TYPE_DIRECTORY: + dirs.append(name) + elif info.ftype == TYPE_FILE: + files.append(name) + else: + if os.path.isdir(os.path.join(root, name)): + dirs.append(name) + elif os.path.exists(os.path.join(path, name)): + files.append(name) + + yield root, dirs, files + for d in sorted(dirs, reverse=True): + if not d.startswith('/'): + if not followlinks and os.path.islink(os.path.join(root, d)): + continue + ds.append(os.path.join(root, d)) + + def check_ok(self, path): + if os.path.isdir(path): + return os.path.exists(os.path.join(path, '.masterinfo')) + else: + return os.path.exists(os.path.join(os.path.dirname(path), + '.masterinfo')) + + def open_file(self, path): + return MooseFile(path, self) + + +class ReadableFile(object): + def __init__(self, path): + self.path = path + self.inode = None + self.info = None + self.length = 0 + + def locs(self, i=None): + raise NotImplementedError + + def seek(self, offset, whence=0): + raise NotImplementedError + + def tell(self): + raise NotImplementedError + + def read(self, n): + raise NotImplementedError + + def __iter__(self): + line = b'' + while True: + data = self.read(-1) + if not data: + break + generator = StringIO(data) + assert b'\n' not in line, line + line += next(generator) + if line.endswith(b'\n'): + yield line + line = b'' + ll = list(generator) + if not ll: + continue + + for line in ll[:-1]: + yield line + line = ll[-1] + if line.endswith(b'\n'): + yield line + line = b'' + if line: + yield line + + def close(self): + raise NotImplementedError + + def __enter__(self): + return self + + def __exit__(self, exc_type, exc_val, exc_tb): + self.close() + + +class PosixFile(ReadableFile): + def __init__(self, path): + ReadableFile.__init__(self, path) + st = os.lstat(path) + self.inode = st.st_ino + self.length = st.st_size + self.info = FileInfo(st.st_ino, path, self._get_type(st.st_mode), st.st_mode, + st.st_uid, st.st_gid, st.st_atime, st.st_mtime, + st.st_ctime, st.st_nlink, st.st_size) + self.fp = open(self.path, 'rb', 4096 * 1024) + + def _get_type(self, st_mode): + if stat.S_ISDIR(st_mode): + return 2 + elif stat.S_ISLNK(st_mode): + return 3 + elif stat.S_ISREG(st_mode): + return 1 + + def locs(self, i=None): + return [] + + def seek(self, offset, whence=0): + self.fp.seek(offset, whence) + + def tell(self): + return self.fp.tell() + + def read(self, n): + return self.fp.read(n) + + def close(self): + self.info = None + self.fp.close() + + +class MooseFile(ReadableFile): + def __init__(self, path, fs): + ReadableFile.__init__(self, path) + self.fs = fs + self._load(path) + + def _load(self, path): + proxy = self.fs._find_proxy(path) + st = os.lstat(path) + self.inode = st.st_ino + self.info = proxy.getattr(self.inode) + self.length = self.info.length + self.cscache = {} + self.roff = 0 + self.rbuf = b'' + self.reader = None + self.generator = None + + def get_chunk(self, i): + chunk = self.cscache.get(i) + if not chunk: + proxy = self.fs._find_proxy(self.path) + chunk = proxy.readchunk(self.inode, i) + self.cscache[i] = chunk + return chunk + + def __getstate__(self): + return self.path + + def __setstate__(self, state): + from . import file_manager + self.path = path = state + self.fs = file_manager._get_fs(path) + self._load(path) + + def locs(self, i=None): + if i is None: + n = (self.length - 1) // CHUNKSIZE + 1 + return [[host for host, _ in self.get_chunk(i).addrs] + for i in range(n)] + return [host for host, _ in self.get_chunk(i).addrs] + + def seek(self, offset, whence=0): + if whence == 1: + offset = self.roff + offset + elif whence == 2: + offset = self.length + offset + assert offset >= 0, 'offset should greater than 0' + off = offset - self.roff + if 0 < off < len(self.rbuf): + self.rbuf = self.rbuf[off:] + else: + self.rbuf = b'' + self.reader = None + + self.roff = offset + self.generator = None + + def tell(self): + return self.roff + + def read(self, n): + if n == -1: + if not self.rbuf: + self.fill_buffer() + v = self.rbuf + self.roff += len(v) + self.rbuf = b'' + return v + + buf = [] + while n > 0: + nbuf = len(self.rbuf) + if nbuf >= n: + buf.append(self.rbuf[:n]) + self.rbuf = self.rbuf[n:] + self.roff += n + break + + if nbuf > 0: + buf.append(self.rbuf) + n -= nbuf + self.rbuf = b'' + self.roff += nbuf + + self.fill_buffer() + if not self.rbuf: + break + return b''.join(buf) + + def fill_buffer(self): + if self.reader is None: + if self.roff < self.length: + self.reader = self.chunk_reader(self.roff) + else: + return + try: + self.rbuf = next(self.reader) + except StopIteration: + self.reader = None + self.fill_buffer() + + def chunk_reader(self, roff): + index = roff // CHUNKSIZE + offset = roff % CHUNKSIZE + chunk = self.get_chunk(index) + length = min(self.length - index * CHUNKSIZE, CHUNKSIZE) + if offset > length: + return + + local_ip = socket.gethostbyname(socket.gethostname()) + for i in range(len(chunk.addrs)): + ip, port = chunk.addrs[i] + if ip == local_ip: + if i != 0: + chunk.addrs[0], chunk.addrs[i] = chunk.addrs[i], chunk.addrs[0] + break + + last_exception = None + last_host = None + for host, port in chunk.addrs: + nerror = 0 + while nerror < 2: + try: + for block in read_chunk(host, port, chunk.id, + chunk.version, + length - offset, + offset): + yield block + offset += len(block) + if offset >= length: + return + nerror = 0 + break + except IOError as e: + last_exception = e + last_host = host + logger.debug("fail to read chunk %d of %s from %s, \ + exception: %s", + chunk.id, self.path, host, e) + nerror += 1 + + raise Exception("unexpected path=%s, addrs=%s, " + "start_offset=%d, chunk=%d, " + "curr_offset_in_chunk=%d < length = %d, " + "last exception on host %s: %s" % + (self.path, + chunk.addrs, + roff, index, + offset, length, + last_host, last_exception)) + + def close(self): + self.roff = 0 + self.rbuf = b'' + self.reader = None + self.generator = None diff --git a/dpark/file_manager/mfs_proxy.py b/dpark/file_manager/mfs_proxy.py new file mode 100644 index 00000000..884ea93b --- /dev/null +++ b/dpark/file_manager/mfs_proxy.py @@ -0,0 +1,238 @@ +from __future__ import absolute_import +import os +import grp +import time +import socket +import getpass +from .utils import unpack, pack, uint8, attrToFileInfo, uint64 +from .consts import * +import six +from six.moves import range + +from dpark.utils.log import get_logger + +logger = get_logger(__name__) + + +class ProtocolError(Exception): + pass + + +class Chunk: + + def __init__(self, index, id_, file_length, version, csdata, ele_width=6): + self.index = index + self.id = id_ + self.file_length = file_length + self.length = min(file_length - index * CHUNKSIZE, CHUNKSIZE) + self.version = version + self.ele_width = ele_width + self.addrs = self._parse(csdata) + + def _parse(self, csdata): + return [(socket.inet_ntoa(csdata[i:i + 4]), + unpack("H", csdata[i + 4:i + 6])[0]) + for i in range(0, len(csdata), self.ele_width)] + + def __repr__(self): + return "" % (self.id, self.version, self.length) + + +class ProxyConn(object): + def __init__(self, host, port, version=(0, 0, 0)): + self.host = host + self.port = port + self.version = version + self.uid = os.getuid() + self.gids = [g.gr_gid for g in grp.getgrall() if getpass.getuser() in g.gr_mem] + self.gids.insert(0, os.getgid()) + self.conn = None + self.msgid = 0 + + @classmethod + def get_masterinfo(cls, path): + while path != os.path.sep: + mp = os.path.join(path, ".masterinfo") + try: + stb = os.lstat(mp) + except OSError: + pass + else: + if stb.st_ino in (0x7FFFFFFF, 0x7FFFFFFE) and \ + stb.st_nlink == 1 and \ + stb.st_uid == 0 and \ + stb.st_gid == 0 and \ + stb.st_size in (10, 14): + sz = stb.st_size + with open(mp, 'rb', 0) as f: + proxyinfo = f.read(sz) + if len(proxyinfo) != sz: + raise Exception('fail to read master info from %s' % mp) + ip = socket.inet_ntoa(proxyinfo[:4]) + port, = unpack("H", proxyinfo[4:]) + if stb.st_size > 10: + major_version, mid_version, minor_version = unpack('HBB', proxyinfo[10:]) + if major_version > 1: + minor_version /= 2 + return ip, port, (major_version, mid_version, minor_version) + else: + return ip, port, (0, 0, 0) + path = os.path.dirname(path) + + def recv_full(self, n): + r = b"" + while len(r) < n: + rr = self.conn.recv(n - len(r)) + if not rr: + raise IOError('need %d bytes, got %d', n, len(r)) + r += rr + return r + + def send_full(self, buf): + n = self.conn.send(buf) + while n < len(buf): + sent = self.conn.send(buf[n:]) + if not sent: + raise IOError('Write failed') + n += sent + + def _connect(self): + if self.conn is not None: + return + N = 8 + for i in range(N): + try: + conn = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + conn.connect((self.host, self.port)) + self.conn = conn + return + except socket.error as e: + if i == N - 1: + raise Exception("Fail to connect to mfs proxy %s:%s, %s", self.host, self.port, e) + time.sleep(1.5 ** i) # 1.5**8 = 25.6 + + def _recv_cmd(self, cmd): + d = self.recv_full(12) + rcmd, size, msgid = unpack("III", d) + data = self.recv_full(size - 4) + if rcmd != cmd + 1: + raise ProtocolError("get incorrect cmd (%s \!= %s)" % (rcmd, cmd + 1)) + + if msgid != self.msgid: + raise ProtocolError('get incorrect msgid(%s != %s)' % (msgid, self.msgid)) + return data + + def sendAndReceive(self, cmd, *args): + self.msgid += 1 + msg = pack(cmd, self.msgid, *args) + num_retry = 3 + for i in range(num_retry): + self._connect() + try: + self.send_full(msg) + data = self._recv_cmd(cmd) + return data + except IOError: + self.terminate() + if i == num_retry - 1: + raise + else: + time.sleep(2 ** i * 0.1) + + def terminate(self): + self.conn.close() + self.conn = None + + def getdirplus(self, inode, max_entries=0xFFFFFFFF, nedgeid=0): + flag = GETDIR_FLAG_WITHATTR + if self.version < (2, 0, 0): + ans = self.sendAndReceive(CLTOMA_FUSE_READDIR, inode, self.uid, self.gids[0], uint8(flag)) + else: + gidsize = len(self.gids) + gids = [gid for gid in self.gids] + gids.append(uint8(flag)) + gids.append(max_entries) + gids.append(uint64(nedgeid)) + ans = self.sendAndReceive(CLTOMA_FUSE_READDIR, inode, + self.uid, gidsize, *gids) + p = 0 + infos = {} + # rnedgeid, = unpack('Q', ans[p: p + 8]) + p += 8 + while p < len(ans): + length, = unpack('B', ans[p: p + 1]) + p += 1 + name = ans[p: p + length] + if not six.PY2: + name = name.decode('utf-8') + + p += length + i, = unpack("I", ans[p: p + 4]) + p += 4 + attr = ans[p: p + 35] + infos[name] = attrToFileInfo(i, attr, name, self.version) + p += 35 + return infos + + def getattr(self, inode, opened=0): + if self.version < (1, 6, 28): + ans = self.sendAndReceive(CLTOMA_FUSE_GETATTR, inode, + self.uid, self.gids[0]) + else: + ans = self.sendAndReceive(CLTOMA_FUSE_GETATTR, inode, + uint8(opened), self.uid, self.gids[0]) + return attrToFileInfo(inode, ans[:35], version=self.version) + + def readlink(self, inode): + ans = self.sendAndReceive(CLTOMA_FUSE_READLINK, inode) + length, = unpack('I', ans) + if length + 4 != len(ans): + raise Exception('invalid length') + return ans[4: -1] + + def readchunk(self, inode, index, chunkopflags=0): + """ + // msgid:32 length:64 chunkid:64 version:32 N*[ ip:32 port:16 ] + // msgid:32 protocolid:8 length:64 chunkid:64 version:32 N*[ ip:32 port:16 cs_ver:32 ] + (master and client both versions >= 1.7.32 - protocolid==1) + // msgid:32 protocolid:8 length:64 chunkid:64 version:32 N*[ ip:32 port:16 cs_ver:32 labelmask:32 ] + (master and client both versions >= 3.0.10 - protocolid==2) + """ + + cnt = 0 + while True: + cnt += 1 + if self.version < (3, 0, 4): + ans = self.sendAndReceive(CLTOMA_FUSE_READ_CHUNK, inode, index) + else: + ans = self.sendAndReceive(CLTOMA_FUSE_READ_CHUNK, inode, + index, uint8(chunkopflags)) + n = len(ans) + if n == 1: + from .utils import Error + err = ord(ans) + if err == ERROR_LOCKED: + if cnt < 100: + time.sleep(0.1) + continue + + logger.warning('Waited too long for locked chunk %s:%s', inode, index) + + raise Error(ord(ans)) + + if n < 20: + raise Exception('read chunk invalid length: %s(expected 20 above)' % n) + + # self.version is master`s version, not mfsmount`s + if self.version >= (1, 7, 32) and ((n - 21) % 14 == 0 or (n - 21) % 10 == 0): + protocolid, flength, id_, version = unpack('BQQI', ans) + if protocolid == 2: + assert (n - 21) % 14 == 0, n + return Chunk(index, id_, flength, version, ans[21:], ele_width=14) + elif protocolid == 1: + assert (n - 21) % 10 == 0, n + return Chunk(index, id_, flength, version, ans[21:], ele_width=10) + + assert (n - 20) % 6 == 0, n + flength, id_, version = unpack("QQI", ans) + return Chunk(index, id_, flength, version, ans[20:]) diff --git a/dpark/file_manager/utils.py b/dpark/file_manager/utils.py new file mode 100644 index 00000000..f201f663 --- /dev/null +++ b/dpark/file_manager/utils.py @@ -0,0 +1,170 @@ +from __future__ import absolute_import +import socket +import struct +from contextlib import closing + +from .consts import * +import six + + +def uint8(n): + return struct.pack("B", n) + + +def uint64(n): + return struct.pack("!Q", n) + + +class Error(Exception): + def __init__(self, code): + self.code = code + + def __str__(self): + return strerror(self.code) + + +def pack(cmd, *args): + msg = [] + for a in args: + if isinstance(a, six.integer_types): + msg.append(struct.pack("!I", a)) + elif isinstance(a, bytes): + msg.append(a) + else: + raise TypeError(str(type(a)) + str(a)) + header = struct.pack("!II", cmd, sum(len(i) for i in msg)) + return header + b''.join(msg) + + +def unpack(fmt, buf): + if not fmt.startswith("!"): + fmt = "!" + fmt + return struct.unpack(fmt, buf[:struct.calcsize(fmt)]) + + +class FileInfo: + def __init__(self, inode, name, ftype, mode, uid, gid, + atime, mtime, ctime, nlink, length): + self.inode = inode + self.name = name + self.ftype = self._get_ftype(ftype) + if ftype == TYPE_DIRECTORY: + mode |= S_IFDIR + elif ftype == TYPE_SYMLINK: + mode |= S_IFLNK + elif ftype == TYPE_FILE: + mode |= S_IFREG + self.mode = mode + self.uid = uid + self.gid = gid + self.atime = atime + self.mtime = mtime + self.ctime = ctime + self.nlink = nlink + self.length = length + self.blocks = (length + 511) / 512 + + def __repr__(self): + return ("FileInfo(%s, inode=%d, type=%s, length=%d)" % + (self.name, self.inode, self.ftype, self.length)) + + def is_symlink(self): + return self.ftype == TYPE_SYMLINK + + @staticmethod + def _get_ftype(ftype): + ftype_str_map = {1: TYPE_FILE, 2: TYPE_DIRECTORY, 3: TYPE_SYMLINK, + 4: TYPE_FIFO, 5: TYPE_BLOCKDEV, 6: TYPE_CHARDEV, + 7: TYPE_SOCKET, 8: TYPE_TRASH, 9: TYPE_SUSTAINED} + if ftype in ftype_str_map: + return ftype_str_map[ftype] + return '?' + + +def attrToFileInfo(inode, attrs, name='', version=(0, 0, 0)): + if len(attrs) != 35: + raise Exception('bad length') + if version < (1, 7, 32): + return FileInfo(inode, name, *struct.unpack("!BHIIIIIIQ", attrs)) + else: + return _unpack_to_file_info(inode, attrs, name) + + +def _unpack_to_file_info(inode, attrs, name): + tup = struct.unpack("!BHIIIIIIQ", attrs) + type_mode = tup[1] + ftype = (type_mode & 0xf000) >> 12 + mode = type_mode & 0x0fff + return FileInfo(inode, name, ftype, mode, *(tup[2:])) + + +def read_chunk(host, port, chunkid, version, size, offset=0): + if offset + size > CHUNKSIZE: + raise ValueError("size too large %s > %s" % + (size, CHUNKSIZE - offset)) + + conn = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + conn.settimeout(10) + conn.connect((host, port)) + + with closing(conn): + msg = pack(CLTOCS_READ, uint64(chunkid), version, offset, size) + n = conn.send(msg) + while n < len(msg): + if not n: + raise IOError("write failed") + msg = msg[n:] + n = conn.send(msg) + + def recv(n): + d = conn.recv(n) + while len(d) < n: + nd = conn.recv(n - len(d)) + if not nd: + raise IOError("not enough data") + d += nd + return d + + while size > 0: + cmd, l = unpack("II", recv(8)) + + if cmd == CSTOCL_READ_STATUS: + if l != 9: + raise Exception("readblock: READ_STATUS incorrect message size") + cid, code = unpack("QB", recv(l)) + if cid != chunkid: + raise Exception("readblock; READ_STATUS incorrect chunkid") + conn.close() + return + + elif cmd == CSTOCL_READ_DATA: + if l < 20: + raise Exception("readblock; READ_DATA incorrect message size") + cid, bid, boff, bsize, crc = unpack("QHHII", recv(20)) + if cid != chunkid: + raise Exception("readblock; READ_STATUS incorrect chunkid") + if l != 20 + bsize: + raise Exception("readblock; READ_DATA incorrect message size ") + if bsize == 0: # FIXME + raise Exception("readblock; empty block") + if bid != offset >> 16: + raise Exception("readblock; READ_DATA incorrect block number") + if boff != offset & 0xFFFF: + raise Exception("readblock; READ_DATA incorrect block offset") + breq = 65536 - boff + if size < breq: + breq = size + if bsize != breq: + raise Exception("readblock; READ_DATA incorrect block size") + + while breq > 0: + data = conn.recv(breq) + if not data: + raise IOError("unexpected ending: need %d" % breq) + yield data + breq -= len(data) + + offset += bsize + size -= bsize + else: + raise Exception("readblock; unknown message: %s" % cmd) diff --git a/dpark/hostatus.py b/dpark/hostatus.py new file mode 100644 index 00000000..1be69763 --- /dev/null +++ b/dpark/hostatus.py @@ -0,0 +1,160 @@ +import time +import random +from six.moves import filter +from dpark.utils.log import get_logger + +PURGE_ELAPSED = 60 * 5 +FAILED_TIMES = 2 +logger = get_logger(__name__) + + +class HostStatus: + def __init__(self, hostname, purge_elapsed=PURGE_ELAPSED): + self.hostname = hostname + self.failed_log = [] + self.succeeded_log = [] + self.failed_cnt = 0 + self.succeed_cnt = 0 + self.start_point = 0 + self.failed_tasks = {} + self.purge_elapsed = purge_elapsed + + def task_succeed(self, task_id): + self.succeeded_log.append(time.time()) + if task_id in self.failed_tasks: + del self.failed_tasks[task_id] + + def task_failed(self, task_id): + cur_ts = time.time() + self.failed_log.append(cur_ts) + if task_id in self.failed_tasks: + self.failed_tasks[task_id].append(cur_ts) + else: + self.failed_tasks[task_id] = [cur_ts] + + def purge_old(self): + cur_ts = time.time() + prev_ts = cur_ts - self.purge_elapsed + self.failed_log = list(filter(lambda x: x > prev_ts, self.failed_log)) + self.succeeded_log = list(filter(lambda x: x > prev_ts, + self.succeeded_log)) + self.failed_cnt = len(self.failed_log) + self.succeed_cnt = len(self.succeeded_log) + self.start_point = self._begin_log_ts() + + def recent_succeed_rate(self): + self.purge_old() + if self.failed_cnt + self.succeed_cnt < 1: + return 1 + return self.succeed_cnt * 1.0 / (self.succeed_cnt + self.failed_cnt) + + def failed_on(self, task_id): + return task_id in self.failed_tasks + + def should_forbit(self, task_id): + self.purge_old() + if task_id in self.failed_tasks: + cur_elapsed = time.time() - self.failed_tasks[task_id][-1] + mask_elapsed = self.purge_elapsed * pow(2, len(self.failed_tasks[task_id])) + return cur_elapsed < mask_elapsed + + return False + + def _begin_log_ts(self): + ts = [time.time()] + if self.failed_log: + ts.append(self.failed_log[0]) + if self.succeeded_log: + ts.append(self.succeeded_log[0]) + return min(ts) + + def total_recent_task_run(self): + return self.succeed_cnt + self.failed_cnt + + def erase_failed_task(self, task_id): + if task_id in self.failed_tasks: + del self.failed_tasks[task_id] + + +class TaskHostManager: + def __init__(self): + self.host_dict = {} + self.task_host_failed_dict = {} + + def register_host(self, hostname, purge_elapsed=PURGE_ELAPSED): + if hostname not in self.host_dict: + # logger.debug('register %s to the task host manager', hostname) + self.host_dict[hostname] = HostStatus(hostname, purge_elapsed=purge_elapsed) + + def task_failed_on_host(self, task_id, host): + if host in self.host_dict: + host_status = self.host_dict[host] + return host_status.failed_on(task_id) + return False + + def offer_choice(self, tid, host_offers, blacklist): + ordi_hosts = [] + fail_hosts = [] + forbit_host = [] + for host in host_offers: + host_status = self.host_dict[host] + if host in blacklist or host_status.should_forbit(tid): + forbit_host.append(host) + elif self.task_failed_on_host(tid, host): + fail_hosts.append((host, host_status.recent_succeed_rate())) + else: + ordi_hosts.append((host, host_status.recent_succeed_rate())) + logger.debug('split the offer in to three parts \n ' + 'ordinary %s \nonce failed %s blacklist host %s', + str(ordi_hosts), str(fail_hosts), str(forbit_host)) + if ordi_hosts: + return host_offers[self._random_weighted_choice(ordi_hosts)] + elif fail_hosts: + return host_offers[self._random_weighted_choice(fail_hosts)] + return None, None + + @staticmethod + def _random_weighted_choice(w_list): + total = sum(w for h, w in w_list) + chosen_w = random.uniform(0, total) + cur_w = 0 + for h, w in w_list: + if cur_w + w >= chosen_w: + return h + cur_w += w + assert False, 'Should not get here' + + def task_succeed(self, task_id, hostname, reason): + logger.debug('task %s %s', task_id, str(reason)) + if hostname in self.host_dict: + host_status = self.host_dict[hostname] + host_status.task_succeed(task_id) + if task_id in self.task_host_failed_dict: + for host in self.task_host_failed_dict[task_id]: + self.host_dict[host].erase_failed_task(task_id) + logger.debug('the failed hosts %s for task %s', + str(self.task_host_failed_dict[task_id]), task_id) + del self.task_host_failed_dict[task_id] + + def task_failed(self, task_id, hostname, reason): + logger.debug('task %s failed with message %s', task_id, str(reason)) + if hostname in self.host_dict: + host_status = self.host_dict[hostname] + host_status.task_failed(task_id) + if task_id not in self.task_host_failed_dict: + self.task_host_failed_dict[task_id] = set() + self.task_host_failed_dict[task_id].add(hostname) + + def is_unhealthy_host(self, host): + if host not in self.host_dict: + return False + host_status = self.host_dict[host] + succeed_rate = host_status.recent_succeed_rate() + duration = time.time() - host_status.start_point + total_tasks = host_status.total_recent_task_run() + if duration > 30 and total_tasks > 20 and succeed_rate < 0.1: + logger.debug('the host %s will be judge unhealthy for ' + 'succeed rate %.1f%% with %d tasks in ' + 'duration more than %.3fs', + host, succeed_rate, total_tasks, duration) + return True diff --git a/dpark/job.py b/dpark/job.py deleted file mode 100644 index 53da1dcc..00000000 --- a/dpark/job.py +++ /dev/null @@ -1,339 +0,0 @@ -import time -import sys -import logging -import socket -from operator import itemgetter - -logger = logging.getLogger(__name__) - -TASK_STARTING = 0 -TASK_RUNNING = 1 -TASK_FINISHED = 2 -TASK_FAILED = 3 -TASK_KILLED = 4 -TASK_LOST = 5 - -def readable(size): - units = ['B', 'KB', 'MB', 'GB', 'TB', 'PB'] - unit = 0 - while size > 1024: - size /= 1024.0 - unit += 1 - return '%.1f%s' % (size, units[unit]) - -class Job: - def __init__(self): - self.id = self.newJobId() - self.start = time.time() - - def slaveOffer(self, s, availableCpus): - raise NotImplementedError - - def statusUpdate(self, t): - raise NotImplementedError - - def error(self, code, message): - raise NotImplementedError - - nextJobId = 0 - @classmethod - def newJobId(cls): - cls.nextJobId += 1 - return cls.nextJobId - -LOCALITY_WAIT = 0 -WAIT_FOR_RUNNING = 10 -MAX_TASK_FAILURES = 4 -MAX_TASK_MEMORY = 15 << 10 # 15GB - -# A Job that runs a set of tasks with no interdependencies. -class SimpleJob(Job): - - def __init__(self, sched, tasks, cpus=1, mem=100): - Job.__init__(self) - self.sched = sched - self.tasks = tasks - - for t in tasks: - t.status = None - t.tried = 0 - t.used = 0 - t.cpus = cpus - t.mem = mem - - self.launched = [False] * len(tasks) - self.finished = [False] * len(tasks) - self.numFailures = [0] * len(tasks) - self.blacklist = [[] for i in xrange(len(tasks))] - self.tidToIndex = {} - self.numTasks = len(tasks) - self.tasksLaunched = 0 - self.tasksFinished = 0 - self.total_used = 0 - - self.lastPreferredLaunchTime = time.time() - - self.pendingTasksForHost = {} - self.pendingTasksWithNoPrefs = [] - self.allPendingTasks = [] - - self.reasons = set() - self.failed = False - self.causeOfFailure = "" - self.last_check = 0 - - for i in range(len(tasks)): - self.addPendingTask(i) - self.host_cache = {} - - @property - def taskEverageTime(self): - if not self.tasksFinished: - return 10 - return max(self.total_used / self.tasksFinished, 5) - - def addPendingTask(self, i): - loc = self.tasks[i].preferredLocations() - if not loc: - self.pendingTasksWithNoPrefs.append(i) - else: - for host in loc: - self.pendingTasksForHost.setdefault(host, []).append(i) - self.allPendingTasks.append(i) - - def getPendingTasksForHost(self, host): - try: - return self.host_cache[host] - except KeyError: - v = self._getPendingTasksForHost(host) - self.host_cache[host] = v - return v - - def _getPendingTasksForHost(self, host): - try: - h, hs, ips = socket.gethostbyname_ex(host) - except Exception: - h, hs, ips = host, [], [] - tasks = sum((self.pendingTasksForHost.get(h, []) - for h in [h] + hs + ips), []) - st = {} - for t in tasks: - st[t] = st.get(t, 0) + 1 - ts = sorted(st.items(), key=itemgetter(1), reverse=True) - return [t for t,_ in ts ] - - def findTaskFromList(self, l, host, cpus, mem): - for i in l: - if self.launched[i] or self.finished[i]: - continue - if host in self.blacklist[i]: - continue - t = self.tasks[i] - if t.cpus <= cpus+1e-4 and t.mem <= mem: - return i - - def findTask(self, host, localOnly, cpus, mem): - localTask = self.findTaskFromList(self.getPendingTasksForHost(host), host, cpus, mem) - if localTask is not None: - return localTask, True - noPrefTask = self.findTaskFromList(self.pendingTasksWithNoPrefs, host, cpus, mem) - if noPrefTask is not None: - return noPrefTask, True - if not localOnly: - return self.findTaskFromList(self.allPendingTasks, host, cpus, mem), False -# else: -# print repr(host), self.pendingTasksForHost - return None, False - - # Respond to an offer of a single slave from the scheduler by finding a task - def slaveOffer(self, host, availableCpus=1, availableMem=100): - now = time.time() - localOnly = (now - self.lastPreferredLaunchTime < LOCALITY_WAIT) - i, preferred = self.findTask(host, localOnly, availableCpus, availableMem) - if i is not None: - task = self.tasks[i] - task.status = TASK_STARTING - task.start = now - task.host = host - task.tried += 1 - prefStr = preferred and "preferred" or "non-preferred" - logger.debug("Starting task %d:%d as TID %s on slave %s (%s)", - self.id, i, task, host, prefStr) - self.tidToIndex[task.id] = i - self.launched[i] = True - self.tasksLaunched += 1 - self.blacklist[i].append(host) - if preferred: - self.lastPreferredLaunchTime = now - return task - logger.debug("no task found %s", localOnly) - - def statusUpdate(self, tid, tried, status, reason=None, result=None, update=None): - logger.debug("job status update %s %s %s", tid, status, reason) - if tid not in self.tidToIndex: - logger.error("invalid tid: %s", tid) - return - i = self.tidToIndex[tid] - if self.finished[i]: - if status == TASK_FINISHED: - logger.debug("Task %d is already finished, ignore it", tid) - return - - task = self.tasks[i] - task.status = status - # when checking, task been masked as not launched - if not self.launched[i]: - self.launched[i] = True - self.tasksLaunched += 1 - - if status == TASK_FINISHED: - self.taskFinished(tid, tried, result, update) - elif status in (TASK_LOST, TASK_FAILED, TASK_KILLED): - self.taskLost(tid, tried, status, reason) - task.start = time.time() - - def taskFinished(self, tid, tried, result, update): - i = self.tidToIndex[tid] - self.finished[i] = True - self.tasksFinished += 1 - task = self.tasks[i] - task.used += time.time() - task.start - self.total_used += task.used - if sys.stderr.isatty(): - title = "Job %d: task %s finished in %.1fs (%d/%d) " % (self.id, tid, - task.used, self.tasksFinished, self.numTasks) - logger.info("Task %s finished in %.1fs (%d/%d) \x1b]2;%s\x07\x1b[1A", - tid, task.used, self.tasksFinished, self.numTasks, title) - - from dpark.schedule import Success - self.sched.taskEnded(task, Success(), result, update) - - for t in range(task.tried): - if t + 1 != tried: - self.sched.killTask(self.id, task.id, t + 1) - - if self.tasksFinished == self.numTasks: - ts = [t.used for t in self.tasks] - tried = [t.tried for t in self.tasks] - logger.info("Job %d finished in %.1fs: min=%.1fs, avg=%.1fs, max=%.1fs, maxtry=%d", - self.id, time.time()-self.start, - min(ts), sum(ts)/len(ts), max(ts), max(tried)) - from dpark.accumulator import LocalReadBytes, RemoteReadBytes - lb, rb = LocalReadBytes.reset(), RemoteReadBytes.reset() - if rb > 0: - logger.info("read %s (%d%% localized)", - readable(lb+rb), lb*100/(rb+lb)) - - self.sched.jobFinished(self) - - def taskLost(self, tid, tried, status, reason): - index = self.tidToIndex[tid] - - from dpark.schedule import FetchFailed - if isinstance(reason, FetchFailed) and self.numFailures[index] >= 1: - logger.warning("Task %s was Lost due to fetch failure from %s", - tid, reason.serverUri) - self.sched.taskEnded(self.tasks[index], reason, None, None) - # cancel tasks - if not self.finished[index]: - self.finished[index] = True - self.tasksFinished += 1 - for i in range(len(self.finished)): - if not self.launched[i]: - self.launched[i] = True - self.tasksLaunched += 1 - self.finished[i] = True - self.tasksFinished += 1 - if self.tasksFinished == self.numTasks: - self.sched.jobFinished(self) # cancel job - return - - task = self.tasks[index] - if status == TASK_KILLED: - task.mem = min(task.mem * 2, MAX_TASK_MEMORY) - for i,t in enumerate(self.tasks): - if not self.launched[i]: - t.mem = max(task.mem, t.mem) - - elif status == TASK_FAILED: - _logger = logger.error if self.numFailures[index] == MAX_TASK_FAILURES\ - else logger.warning - if reason not in self.reasons: - _logger("task %s failed @ %s: %s\n%s", task.id, task.host, task, reason) - self.reasons.add(reason) - else: - _logger("task %s failed @ %s: %s", task.id, task.host, task) - - elif status == TASK_LOST: - logger.warning("Lost Task %d (task %d:%d:%s) %s at %s", index, self.id, - tid, tried, reason, task.host) - - self.numFailures[index] += 1 - if self.numFailures[index] > MAX_TASK_FAILURES: - logger.error("Task %d failed more than %d times; aborting job", - index, MAX_TASK_FAILURES) - self.abort("Task %d failed more than %d times" - % (index, MAX_TASK_FAILURES)) - - self.launched[index] = False - if self.tasksLaunched == self.numTasks: - self.sched.requestMoreResources() - for i in xrange(len(self.blacklist)): - self.blacklist[i] = [] - self.tasksLaunched -= 1 - - def check_task_timeout(self): - now = time.time() - if self.last_check + 5 > now: - return False - self.last_check = now - - n = self.launched.count(True) - if n != self.tasksLaunched: - logger.warning("bug: tasksLaunched(%d) != %d", self.tasksLaunched, n) - self.tasksLaunched = n - - for i in xrange(self.numTasks): - task = self.tasks[i] - if (self.launched[i] and task.status == TASK_STARTING - and task.start + WAIT_FOR_RUNNING < now): - logger.debug("task %d timeout %.1f (at %s), re-assign it", - task.id, now - task.start, task.host) - self.launched[i] = False - self.tasksLaunched -= 1 - - if self.tasksFinished > self.numTasks * 2.0 / 3: - scale = 1.0 * self.numTasks / self.tasksFinished - avg = max(self.taskEverageTime, 10) - tasks = sorted((task.start, i, task) - for i,task in enumerate(self.tasks) - if self.launched[i] and not self.finished[i]) - for _t, idx, task in tasks: - used = now - task.start - #logger.debug("task %s used %.1f (avg = %.1f)", task.id, used, avg) - if used > avg * (2 ** task.tried) * scale: - # re-submit timeout task - if task.tried <= MAX_TASK_FAILURES: - logger.debug("re-submit task %s for timeout %.1f, try %d", - task.id, used, task.tried) - task.used += used - task.start = now - self.launched[idx] = False - self.tasksLaunched -= 1 - else: - logger.error("task %s timeout, aborting job %s", - task, self.id) - self.abort("task %s timeout" % task) - else: - break - return self.tasksLaunched < n - - def abort(self, message): - logger.error("abort the job: %s", message) - tasks = ' '.join(str(i) for i in xrange(len(self.finished)) - if not self.finished[i]) - logger.error("not finished tasks: %s", tasks) - self.failed = True - self.causeOfFailure = message - self.sched.jobFinished(self) - self.sched.shutdown() diff --git a/dpark/moosefs/__init__.py b/dpark/moosefs/__init__.py deleted file mode 100644 index e3b40897..00000000 --- a/dpark/moosefs/__init__.py +++ /dev/null @@ -1,360 +0,0 @@ -import os -import socket -from cStringIO import StringIO -import logging - -from consts import * -from master import MasterConn -from cs import read_chunk, read_chunk_from_local - -MFS_ROOT_INODE = 1 - -logger = logging.getLogger(__name__) - -class CrossSystemSymlink(Exception): - def __init__(self, src, dst): - self.src = src - self.dst = dst - def __str__(self): - return '%s -> %s' % (self.src, self.dst) - -class MooseFS(object): - def __init__(self, host='mfsmaster', port=9421, mountpoint='/mfs'): - self.host = host - self.mountpoint = mountpoint - self.mc = MasterConn(host, port) - self.symlink_cache = {} - - def _lookup(self, parent, name): - return self.mc.lookup(parent, name)[0] - - def readlink(self, inode): - target = self.symlink_cache.get(inode) - if target is None: - target = self.mc.readlink(inode) - self.symlink_cache[inode] = target - return target - - def lookup(self, path, followSymlink=True): - parent = MFS_ROOT_INODE - info = None - ps = path.split('/') - for i, n in enumerate(ps): - if not n: continue - info = self._lookup(parent, n) - if not info: - return - while info.is_symlink() and followSymlink: - target = self.readlink(info.inode) - if not target.startswith('/'): - target = os.path.join('/'.join(ps[:i]), target) - info = self.lookup(target, followSymlink) - elif target.startswith(self.mountpoint): - info = self.lookup(target[len(self.mountpoint):], followSymlink) - else: - raise CrossSystemSymlink(path, os.path.join(target, *ps[i+1:])) - parent = info.inode - if info is None and parent == MFS_ROOT_INODE: - info = self.mc.getattr(parent) - return info - - def open(self, path): - info = self.lookup(path) - if not info: - raise Exception("not found") - return File(info.inode, path, info, self.host) - - def listdir(self, path): - info = self.lookup(path) - if not info: - raise Exception("not found") - return self.mc.getdirplus(info.inode) - - def walk(self, path, followlinks=False): - ds = [path] - while ds: - root = ds.pop() - cs = self.listdir(root) - dirs, files = [], [] - for name, info in cs.iteritems(): - if name in '..': continue - while followlinks and info and info.type == TYPE_SYMLINK: - target = self.readlink(info.inode) - if target.startswith('/'): - if not target.startswith(self.mountpoint): - if os.path.exists(target): - if os.path.isdir(target): - dirs.append(target) - else: - files.append(target) - info = None # ignore broken symlink - break - else: - target = target[len(self.mountpoint):] - # use relative path for internal symlinks - name = ('../' * len(filter(None, root.split('/')))) + target - else: - name = target - target = os.path.join(root, target) - info = self.lookup(target) - - if info: - if info.type == TYPE_DIRECTORY: - if name not in dirs: - dirs.append(name) - elif info.type == TYPE_FILE: - if name not in files: - files.append(name) - - yield root, dirs, files - for d in sorted(dirs, reverse=True): - if not d.startswith('/'): # skip external links - ds.append(os.path.join(root, d)) - - def close(self): - self.mc.close() - - -class File(object): - def __init__(self, inode, path, info, master): - self.inode = inode - self.path = path - self.info = info - self.length = info.length - self.master = master - self.cscache = {} - - def get_chunk(self, i): - chunk = self.cscache.get(i) - if not chunk: - chunk = get_mfs(self.master).mc.readchunk(self.inode, i) - self.cscache[i] = chunk - return chunk - - def locs(self, i=None): - if i is None: - n = (self.length - 1) / CHUNKSIZE + 1 - return [[host for host, _ in self.get_chunk(i).addrs] - for i in range(n)] - return [host for host, _ in self.get_chunk(i).addrs] - - -class ReadableFile(File): - def __init__(self, f): - self.__dict__.update(f.__dict__) - self.roff = 0 - self.rbuf = '' - self.reader = None - self.generator = None - - def seek(self, offset, whence=0): - if whence == 1: - offset = self.roff + offset - elif whence == 2: - offset = self.length + offset - assert offset >= 0, 'offset should greater than 0' - - off = offset - self.roff - if off > 0 and off < len(self.rbuf): - self.rbuf = self.rbuf[off:] - else: - self.rbuf = '' - self.reader = None - - self.roff = offset - self.generator = None - - def tell(self): - return self.roff - - def read(self, n): - if n == -1: - if not self.rbuf: - self.fill_buffer() - v = self.rbuf - self.roff += len(v) - self.rbuf = '' - return v - - buf = [] - while n > 0: - nbuf = len(self.rbuf) - if nbuf >= n: - buf.append(self.rbuf[:n]) - self.rbuf = self.rbuf[n:] - self.roff += n - break - - if nbuf > 0: - buf.append(self.rbuf) - n -= nbuf - self.rbuf = '' - self.roff += nbuf - - self.fill_buffer() - if not self.rbuf: - break - return ''.join(buf) - - def fill_buffer(self): - if self.reader is None: - if self.roff < self.length: - self.reader = self.chunk_reader(self.roff) - else: - return - try: - self.rbuf = self.reader.next() - except StopIteration: - self.reader = None - self.fill_buffer() - - def chunk_reader(self, roff): - index = roff / CHUNKSIZE - offset = roff % CHUNKSIZE - chunk = self.get_chunk(index) - length = min(self.length - index * CHUNKSIZE, CHUNKSIZE) - if offset > length: - return - - local_ip = socket.gethostbyname(socket.gethostname()) - if any(ip == local_ip for ip,port in chunk.addrs): - try: - for block in read_chunk_from_local(chunk.id, - chunk.version, length-offset, offset): - yield block - offset += len(block) - if offset >= length: - return - except Exception, e: - logger.warning("read chunk %d from local: %s", chunk.id, e) - - for host, port in chunk.addrs: - # give up after two continuous errors - nerror = 0 - while nerror < 2: - try: - for block in read_chunk(host, port, chunk.id, - chunk.version, length-offset, offset): - yield block - offset += len(block) - if offset >= length: - return - nerror = 0 - break - except IOError, e: - #print 'read chunk error from ', host, port, chunk.id, chunk.version, offset, e - nerror += 1 - - raise Exception("unexpected error: %d %d %s < %s" % (roff, index, offset, length)) - - def __iter__(self): - # TODO: speedup - line = "" - while True: - data = self.read(-1) - if not data: - break - generator = StringIO(data) - assert '\n' not in line, line - line += generator.next() - if line.endswith('\n'): - yield line - line = '' - - ll = list(generator) - if not ll: continue - - for line in ll[:-1]: - yield line - line = ll[-1] - if line.endswith('\n'): - yield line - line = '' - - if line: - yield line - - def close(self): - self.roff = 0 - self.rbuf = '' - self.reader = None - self.generator = None - - def __enter__(self): - return self - - def __exit__(self, type, value, tb): - self.close() - -_mfs = {} - -MFS_PREFIX = { - } - -def get_mfs(master, mountpoint=''): - if master in _mfs: - return _mfs[master] - _mfs[master] = MooseFS(master, mountpoint=mountpoint) - return _mfs[master] - -def mfsopen(path, master='mfsmaster'): - return get_mfs(master).open(path) - -def listdir(path, master='mfsmaster'): - return get_mfs(master).listdir(path) - -def get_mfs_by_path(path): - for prefix, master in MFS_PREFIX.iteritems(): - if path.startswith(prefix): - return get_mfs(master, prefix) - -def add_prefix(gen, prefix): - for root, dirs, names in gen: - yield prefix + root, dirs, names - for d in dirs: - if d.startswith('/'): - for root, dd, ns in walk(d, True): - yield root, dd, ns - -def walk(path, followlinks=False): - path = os.path.realpath(path) - mfs = get_mfs_by_path(path) - if mfs: - rs = mfs.walk(path[len(mfs.mountpoint):], followlinks) - return add_prefix(rs, mfs.mountpoint) - else: - return os.walk(path, followlinks=followlinks) - -def open_file(path): - mfs = get_mfs_by_path(path) - if mfs: - try: - return mfs.open(path[len(mfs.mountpoint):]) - except CrossSystemSymlink, e: - return open_file(e.dst) - -def _test(): - f = open('/mfs2/test.csv') - f.seek(1024) - d = f.read(1024) - - f2 = mfsopen('/test.csv') - print 'f2 locs', f2.locs() - f2.seek(1024) - d2 = f2.read(1024) - assert d == d2 - - f.seek(0) - f2.seek(0) - import csv - for _ in csv.reader(f2): - break - - #print listdir('/') - for root, dirs, names in walk('/'): - print root, dirs, names - for n in names: - print n, mfsopen(os.path.join(root, n)).locs() - -if __name__ == '__main__': - _test() diff --git a/dpark/moosefs/consts.py b/dpark/moosefs/consts.py deleted file mode 100644 index e5fc0f9e..00000000 --- a/dpark/moosefs/consts.py +++ /dev/null @@ -1,489 +0,0 @@ - -VERSION_ANY = 0 -CRC_POLY = 0xEDB88320 -MFS_ROOT_ID = 1 -MFS_NAME_MAX = 255 -MFS_MAX_FILE_SIZE = 0x20000000000 - -# 1.6.21 -VERSION = 0x010615 - -CHUNKSIZE = 1<<26 - -GETDIR_FLAG_WITHATTR = 0x01 -GETDIR_FLAG_ADDTOCACHE = 0x02 -GETDIR_FLAG_DIRCACHE = 0x04 - -#type for readdir command -TYPE_FILE = 'f' -TYPE_SYMLINK = 'l' -TYPE_DIRECTORY = 'd' -TYPE_FIFO = 'q' -TYPE_BLOCKDEV = 'b' -TYPE_CHARDEV = 'c' -TYPE_SOCKET = 's' -TYPE_TRASH = 't' -TYPE_RESERVED = 'r' -TYPE_UNKNOWN = '?' - -# status code -STATUS_OK = 0 # OK - -ERROR_EPERM = 1 # Operation not permitted -ERROR_ENOTDIR = 2 # Not a directory -ERROR_ENOENT = 3 # No such file or directory -ERROR_EACCES = 4 # Permission denied -ERROR_EEXIST = 5 # File exists -ERROR_EINVAL = 6 # Invalid argument -ERROR_ENOTEMPTY = 7 # Directory not empty -ERROR_CHUNKLOST = 8 # Chunk lost -ERROR_OUTOFMEMORY = 9 # Out of memory - -ERROR_INDEXTOOBIG = 10 # Index too big -ERROR_LOCKED = 11 # Chunk locked -ERROR_NOCHUNKSERVERS = 12 # No chunk servers -ERROR_NOCHUNK = 13 # No such chunk -ERROR_CHUNKBUSY = 14 # Chunk is busy -ERROR_REGISTER = 15 # Incorrect register BLOB -ERROR_NOTDONE = 16 # None of chunk servers performed requested operation -ERROR_NOTOPENED = 17 # File not opened -ERROR_NOTSTARTED = 18 # Write not started - -ERROR_WRONGVERSION = 19 # Wrong chunk version -ERROR_CHUNKEXIST = 20 # Chunk already exists -ERROR_NOSPACE = 21 # No space left -ERROR_IO = 22 # IO error -ERROR_BNUMTOOBIG = 23 # Incorrect block number -ERROR_WRONGSIZE = 24 # Incorrect size -ERROR_WRONGOFFSET = 25 # Incorrect offset -ERROR_CANTCONNECT = 26 # Can't connect -ERROR_WRONGCHUNKID = 27 # Incorrect chunk id -ERROR_DISCONNECTED = 28 # Disconnected -ERROR_CRC = 29 # CRC error -ERROR_DELAYED = 30 # Operation delayed -ERROR_CANTCREATEPATH = 31 # Can't create path - -ERROR_MISMATCH = 32 # Data mismatch -ERROR_EROFS = 33 # Read-only file system -ERROR_QUOTA = 34 # Quota exceeded -ERROR_BADSESSIONID = 35 # Bad session id -ERROR_NOPASSWORD = 36 # Password is needed -ERROR_BADPASSWORD = 37 # Incorrect password -ERROR_MAX = 38 - -# flags: "flags" fileld in "CUTOMA_FUSE_AQUIRE" -WANT_READ = 1 -WANT_WRITE = 2 -AFTER_CREATE = 4 - -# flags: "setmask" field in "CUTOMA_FUSE_SETATTR" -# SET_GOAL_FLAG,SET_DELETE_FLAG are no longer supported -# SET_LENGTH_FLAG,SET_OPENED_FLAG are deprecated -# instead of using FUSE_SETATTR with SET_GOAL_FLAG use FUSE_SETGOAL command -# instead of using FUSE_SETATTR with SET_GOAL_FLAG use FUSE_SETTRASH_TIMEOUT command -# instead of using FUSE_SETATTR with SET_LENGTH_FLAG/SET_OPENED_FLAG use FUSE_TRUNCATE command -SET_GOAL_FLAG = 1 << 0 -SET_MODE_FLAG = 1 << 1 -SET_UID_FLAG = 1 << 2 -SET_GID_FLAG = 1 << 3 -SET_LENGTH_FLAG = 1 << 4 -SET_MTIME_FLAG = 1 << 5 -SET_ATIME_FLAG = 1 << 6 -SET_OPENED_FLAG = 1 << 7 -SET_DELETE_FLAG = 1 << 8 -ANTOAN_NOP = 0 - -# CHUNKSERVER <-> CLIENT/CHUNKSERVER -CUTOCS_READ = 200 -# chunkid:64 version:32 offset:32 size:32 -CSTOCU_READ_STATUS = 201 -# chunkid:64 status:8 -CSTOCU_READ_DATA = 202 -# chunkid:64 blocknum:16 offset:16 size:32 crc:32 size*[ databyte:8 ] - -CUTOCS_WRITE = 210 -# chunkid:64 version:32 N*[ ip:32 port:16 ] -CSTOCU_WRITE_STATUS = 211 -# chunkid:64 writeid:32 status:8 -CUTOCS_WRITE_DATA = 212 -# chunkid:64 writeid:32 blocknum:16 offset:16 size:32 crc:32 size*[ databyte:8 ] -CUTOCS_WRITE_FINISH = 213 -# chunkid:64 version:32 - -#ANY <-> CHUNKSERVER -ANTOCS_CHUNK_CHECKSUM = 300 -# chunkid:64 version:32 -CSTOAN_CHUNK_CHECKSUM = 301 -# chunkid:64 version:32 checksum:32 -# chunkid:64 version:32 status:8 - -ANTOCS_CHUNK_CHECKSUM_TAB = 302 -# chunkid:64 version:32 -CSTOAN_CHUNK_CHECKSUM_TAB = 303 -# chunkid:64 version:32 1024*[checksum:32] -# chunkid:64 version:32 status:8 - -# CLIENT <-> MASTER - -# old attr record: -# type:8 flags:8 mode:16 uid:32 gid:32 atime:32 mtime:32 ctime:32 length:64 -# total: 32B (1+1+2+4+4+4+4+4+8 -# -# flags: ---DGGGG -# |\--/ -# | \------ goal -# \--------- delete imediatelly - -# new attr record: -# type:8 mode:16 uid:32 gid:32 atime:32 mtime:32 ctime:32 nlink:32 length:64 -# total: 35B -# -# mode: FFFFMMMMMMMMMMMM -# \--/\----------/ -# \ \------- mode -# \-------------- flags -# -# in case of BLOCKDEV and CHARDEV instead of 'length:64' on the end there is 'mojor:16 minor:16 empty:32' - -# NAME type: -# ( leng:8 data:lengB - - -FUSE_REGISTER_BLOB_NOACL = "kFh9mdZsR84l5e675v8bi54VfXaXSYozaU3DSz9AsLLtOtKipzb9aQNkxeOISx64" -# CUTOMA: -# clientid:32 [ version:32 ] -# MATOCU: -# clientid:32 -# status:8 - -FUSE_REGISTER_BLOB_TOOLS_NOACL = "kFh9mdZsR84l5e675v8bi54VfXaXSYozaU3DSz9AsLLtOtKipzb9aQNkxeOISx63" -# CUTOMA: -# - -# MATOCU: -# status:8 - -FUSE_REGISTER_BLOB_ACL = "DjI1GAQDULI5d2YjA26ypc3ovkhjvhciTQVx3CS4nYgtBoUcsljiVpsErJENHaw0" - -REGISTER_GETRANDOM = 1 -# rcode==1: generate random blob -# CUTOMA: -# rcode:8 -# MATOCU: -# randomblob:32B - -REGISTER_NEWSESSION = 2 -# rcode==2: first register -# CUTOMA: -# rcode:8 version:32 ileng:32 info:ilengB pleng:32 path:plengB [ passcode:16B ] -# MATOCU: -# sessionid:32 sesflags:8 rootuid:32 rootgid:32 -# status:8 - -REGISTER_RECONNECT = 3 -# rcode==3: mount reconnect -# CUTOMA: -# rcode:8 sessionid:32 version:32 -# MATOCU: -# status:8 - -REGISTER_TOOLS = 4 -# rcode==4: tools connect -# CUTOMA: -# rcode:8 sessionid:32 version:32 -# MATOCU: -# status:8 - -REGISTER_NEWMETASESSION = 5 -# rcode==5: first register -# CUTOMA: -# rcode:8 version:32 ileng:32 info:ilengB [ passcode:16B ] -# MATOCU: -# sessionid:32 sesflags:8 -# status:8 - -CUTOMA_FUSE_REGISTER = 400 -# blob:64B ... (depends on blob - see blob descriptions above -MATOCU_FUSE_REGISTER = 401 -# depends on blob - see blob descriptions above -CUTOMA_FUSE_STATFS = 402 -# msgid:32 - -MATOCU_FUSE_STATFS = 403 -# msgid:32 totalspace:64 availspace:64 trashspace:64 inodes:32 -CUTOMA_FUSE_ACCESS = 404 -# msgid:32 inode:32 uid:32 gid:32 modemask:8 -MATOCU_FUSE_ACCESS = 405 -# msgid:32 status:8 -CUTOMA_FUSE_LOOKUP = 406 -# msgid:32 inode:32 name:NAME uid:32 gid:32 -MATOCU_FUSE_LOOKUP = 407 -# msgid:32 status:8 -# msgid:32 inode:32 attr:35B -CUTOMA_FUSE_GETATTR = 408 -# msgid:32 inode:32 -# msgid:32 inode:32 uid:32 gid:32 -MATOCU_FUSE_GETATTR = 409 -# msgid:32 status:8 -# msgid:32 attr:35B -CUTOMA_FUSE_SETATTR = 410 -# msgid:32 inode:32 uid:32 gid:32 setmask:8 attr:32B - compatibility with very old version -# msgid:32 inode:32 uid:32 gid:32 setmask:16 attr:32B - compatibility with old version -# msgid:32 inode:32 uid:32 gid:32 setmask:8 attrmode:16 attruid:32 attrgid:32 attratime:32 attrmtime:32 -MATOCU_FUSE_SETATTR = 411 -# msgid:32 status:8 -# msgid:32 attr:35B -CUTOMA_FUSE_READLINK = 412 -# msgid:32 inode:32 -MATOCU_FUSE_READLINK = 413 -# msgid:32 status:8 -# msgid:32 length:32 path:lengthB -CUTOMA_FUSE_SYMLINK = 414 -# msgid:32 inode:32 name:NAME length:32 path:lengthB uid:32 gid:32 -MATOCU_FUSE_SYMLINK = 415 -# msgid:32 status:8 -# msgid:32 inode:32 attr:35B -CUTOMA_FUSE_MKNOD = 416 -# msgid:32 inode:32 name:NAME type:8 mode:16 uid:32 gid:32 rdev:32 -MATOCU_FUSE_MKNOD = 417 -# msgid:32 status:8 -# msgid:32 inode:32 attr:35B -CUTOMA_FUSE_MKDIR = 418 -# msgid:32 inode:32 name:NAME mode:16 uid:32 gid:32 -MATOCU_FUSE_MKDIR = 419 -# msgid:32 status:8 -# msgid:32 inode:32 attr:35B -CUTOMA_FUSE_UNLINK = 420 -# msgid:32 inode:32 name:NAME uid:32 gid:32 -MATOCU_FUSE_UNLINK = 421 -# msgid:32 status:8 -CUTOMA_FUSE_RMDIR = 422 -# msgid:32 inode:32 name:NAME uid:32 gid:32 -MATOCU_FUSE_RMDIR = 423 -# msgid:32 status:8 -CUTOMA_FUSE_RENAME = 424 -# msgid:32 inode_src:32 name_src:NAME inode_dst:32 name_dst:NAME uid:32 gid:32 -MATOCU_FUSE_RENAME = 425 -# msgid:32 status:8 -CUTOMA_FUSE_LINK = 426 -# msgid:32 inode:32 inode_dst:32 name_dst:NAME uid:32 gid:32 -MATOCU_FUSE_LINK = 427 -# msgid:32 status:8 -# msgid:32 inode:32 attr:35B -CUTOMA_FUSE_GETDIR = 428 -# msgid:32 inode:32 uid:32 gid:32 - old version (works like new version with flags==0 -# msgid:32 inode:32 uid:32 gid:32 flags:8 -MATOCU_FUSE_GETDIR = 429 -# msgid:32 status:8 -# msgid:32 N*[ name:NAME inode:32 type:8 ] - when GETDIR_FLAG_WITHATTR in flags is not set -# msgid:32 N*[ name:NAME inode:32 type:35B ] - when GETDIR_FLAG_WITHATTR in flags is set -CUTOMA_FUSE_OPEN = 430 -# msgid:32 inode:32 uid:32 gid:32 flags:8 -MATOCU_FUSE_OPEN = 431 -# msgid:32 status:8 -# since 1.6.9 if no error: -# msgid:32 attr:35B - -CUTOMA_FUSE_READ_CHUNK = 432 -# msgid:32 inode:32 chunkindx:32 -MATOCU_FUSE_READ_CHUNK = 433 -# msgid:32 status:8 -# msgid:32 length:64 chunkid:64 version:32 N*[ip:32 port:16] -# msgid:32 length:64 srcs:8 srcs*[chunkid:64 version:32 ip:32 port:16] - not implemented -CUTOMA_FUSE_WRITE_CHUNK = 434 # it creates, duplicates or sets new version of chunk if necessary */ -# msgid:32 inode:32 chunkindx:32 -MATOCU_FUSE_WRITE_CHUNK = 435 -# msgid:32 status:8 -# msgid:32 length:64 chunkid:64 version:32 N*[ip:32 port:16] -CUTOMA_FUSE_WRITE_CHUNK_END = 436 -# msgid:32 chunkid:64 inode:32 length:64 -MATOCU_FUSE_WRITE_CHUNK_END = 437 -# msgid:32 status:8 - - -CUTOMA_FUSE_APPEND = 438 -# msgid:32 inode:32 srcinode:32 uid:32 gid:32 - append to existing element -MATOCU_FUSE_APPEND = 439 -# msgid:32 status:8 - - -CUTOMA_FUSE_CHECK = 440 -# msgid:32 inode:32 -MATOCU_FUSE_CHECK = 441 -# msgid:32 status:8 -# msgid:32 N*[ copies:8 chunks:16 ] - -CUTOMA_FUSE_GETTRASHTIME = 442 -# msgid:32 inode:32 gmode:8 -MATOCU_FUSE_GETTRASHTIME = 443 -# msgid:32 status:8 -# msgid:32 tdirs:32 tfiles:32 tdirs*[ trashtime:32 dirs:32 ] tfiles*[ trashtime:32 files:32 ] -CUTOMA_FUSE_SETTRASHTIME = 444 -# msgid:32 inode:32 uid:32 trashtimeout:32 smode:8 -MATOCU_FUSE_SETTRASHTIME = 445 -# msgid:32 status:8 -# msgid:32 changed:32 notchanged:32 notpermitted:32 - -CUTOMA_FUSE_GETGOAL = 446 -# msgid:32 inode:32 gmode:8 -MATOCU_FUSE_GETGOAL = 447 -# msgid:32 status:8 -# msgid:32 gdirs:8 gfiles:8 gdirs*[ goal:8 dirs:32 ] gfiles*[ goal:8 files:32 ] - -CUTOMA_FUSE_SETGOAL = 448 -# msgid:32 inode:32 uid:32 goal:8 smode:8 -MATOCU_FUSE_SETGOAL = 449 -# msgid:32 status:8 -# msgid:32 changed:32 notchanged:32 notpermitted:32 - -CUTOMA_FUSE_GETTRASH = 450 -# msgid:32 -MATOCU_FUSE_GETTRASH = 451 -# msgid:32 status:8 -# msgid:32 N*[ name:NAME inode:32 ] - -CUTOMA_FUSE_GETDETACHEDATTR = 452 -# msgid:32 inode:32 dtype:8 -MATOCU_FUSE_GETDETACHEDATTR = 453 -# msgid:32 status:8 -# msgid:32 attr:35B - -CUTOMA_FUSE_GETTRASHPATH = 454 -# msgid:32 inode:32 -MATOCU_FUSE_GETTRASHPATH = 455 -# msgid:32 status:8 -# msgid:32 length:32 path:lengthB - -CUTOMA_FUSE_SETTRASHPATH = 456 -# msgid:32 inode:32 length:32 path:lengthB -MATOCU_FUSE_SETTRASHPATH = 457 -# msgid:32 status:8 - -CUTOMA_FUSE_UNDEL = 458 -# msgid:32 inode:32 -MATOCU_FUSE_UNDEL = 459 -# msgid:32 status:8 -CUTOMA_FUSE_PURGE = 460 -# msgid:32 inode:32 -MATOCU_FUSE_PURGE = 461 -# msgid:32 status:8 - -CUTOMA_FUSE_GETDIRSTATS = 462 -# msgid:32 inode:32 -MATOCU_FUSE_GETDIRSTATS = 463 -# msgid:32 status:8 -# msgid:32 inodes:32 dirs:32 files:32 ugfiles:32 mfiles:32 chunks:32 ugchunks:32 mchunks32 length:64 size:64 gsize:64 - -CUTOMA_FUSE_TRUNCATE = 464 -# msgid:32 inode:32 [opened:8] uid:32 gid:32 opened:8 length:64 -MATOCU_FUSE_TRUNCATE = 465 -# msgid:32 status:8 -# msgid:32 attr:35B - -CUTOMA_FUSE_REPAIR = 466 -# msgid:32 inode:32 uid:32 gid:32 -MATOCU_FUSE_REPAIR = 467 -# msgid:32 status:8 -# msgid:32 notchanged:32 erased:32 repaired:32 - -CUTOMA_FUSE_SNAPSHOT = 468 -# msgid:32 inode:32 inode_dst:32 name_dst:NAME uid:32 gid:32 canoverwrite:8 -MATOCU_FUSE_SNAPSHOT = 469 -# msgid:32 status:8 - -CUTOMA_FUSE_GETRESERVED = 470 -# msgid:32 -MATOCU_FUSE_GETRESERVED = 471 -# msgid:32 status:8 -# msgid:32 N*[ name:NAME inode:32 ] - -CUTOMA_FUSE_GETEATTR = 472 -# msgid:32 inode:32 gmode:8 -MATOCU_FUSE_GETEATTR = 473 -# msgid:32 status:8 -# msgid:32 eattrdirs:8 eattrfiles:8 eattrdirs*[ eattr:8 dirs:32 ] eattrfiles*[ eattr:8 files:32 ] - -CUTOMA_FUSE_SETEATTR = 474 -# msgid:32 inode:32 uid:32 eattr:8 smode:8 -MATOCU_FUSE_SETEATTR = 475 -# msgid:32 status:8 -# msgid:32 changed:32 notchanged:32 notpermitted:32 - -CUTOMA_FUSE_QUOTACONTROL = 476 -# msgid:32 inode:32 qflags:8 - delete quota -# msgid:32 inode:32 qflags:8 sinodes:32 slength:64 ssize:64 srealsize:64 hinodes:32 hlength:64 hsize:64 hrealsize:64 - set quota -MATOCU_FUSE_QUOTACONTROL = 477 -# msgid:32 status:8 -# msgid:32 qflags:8 sinodes:32 slength:64 ssize:64 srealsize:64 hinodes:32 hlength:64 hsize:64 hrealsize:64 curinodes:32 curlength:64 cursize:64 currealsize:64 - -CUTOMA_FUSE_DIR_REMOVED = 490 -# msgid:32 N*[ inode:32 ] - -MATOCU_FUSE_NOTIFY_ATTR = 491 -# msgid:32 N*[ parent:32 inode:32 attr:35B ] -MATOCU_FUSE_NOTIFY_DIR = 492 -# msgid:32 N*[ inode:32 ] -# special - reserved (opened) inodes - keep opened files. -CUTOMA_FUSE_RESERVED_INODES = 499 -# N*[inode:32] - -errtab = [ - "OK", - "Operation not permitted", - "Not a directory", - "No such file or directory", - "Permission denied", - "File exists", - "Invalid argument", - "Directory not empty", - "Chunk lost", - "Out of memory", - "Index too big", - "Chunk locked", - "No chunk servers", - "No such chunk", - "Chunk is busy", - "Incorrect register BLOB", - "None of chunk servers performed requested operation", - "File not opened", - "Write not started", - "Wrong chunk version", - "Chunk already exists", - "No space left", - "IO error", - "Incorrect block number", - "Incorrect size", - "Incorrect offset", - "Can't connect", - "Incorrect chunk id", - "Disconnected", - "CRC error", - "Operation delayed", - "Can't create path", - "Data mismatch", - "Read-only file system", - "Quota exceeded", - "Bad session id", - "Password is needed", - "Incorrect password", - "Unknown MFS error", -] - -def mfs_strerror(code): - if code > ERROR_MAX: - code = ERROR_MAX - return errtab[code] - -S_IFMT = 0170000 # type of file */ -S_IFIFO = 0010000 # named pipe (fifo) */ -S_IFCHR = 0020000 # character special */ -S_IFDIR = 0040000 # directory */ -S_IFBLK = 0060000 # block special */ -S_IFREG = 0100000 # regular */ -S_IFLNK = 0120000 # symbolic link */ -S_IFSOCK = 0140000 # socket */ -S_IFWHT = 0160000 # whiteout */ -S_ISUID = 0004000 # set user id on execution */ -S_ISGID = 0002000 # set group id on execution */ -S_ISVTX = 0001000 # save swapped text even after use */ -S_IRUSR = 0000400 # read permission, owner */ -S_IWUSR = 0000200 # write permission, owner */ -S_IXUSR = 0000100 # execute/search permission, owner */ diff --git a/dpark/moosefs/cs.py b/dpark/moosefs/cs.py deleted file mode 100644 index 33844b2a..00000000 --- a/dpark/moosefs/cs.py +++ /dev/null @@ -1,148 +0,0 @@ -import os -import socket -import logging -import commands - -from consts import CHUNKSIZE, CUTOCS_READ, CSTOCU_READ_DATA, CSTOCU_READ_STATUS -from utils import uint64, pack, unpack - -logger = logging.getLogger(__name__) - -mfsdirs = [] -def _scan(): - cmd = """ps -eo cmd| grep mfschunkserver | grep -v grep | - head -1 | cut -d ' ' -f1 | xargs dirname | sed 's#sbin##g'""" - mfs_prefix = commands.getoutput(cmd) - mfs_cfg = '%s/etc/mfshdd.cfg' % mfs_prefix - mfs_cfg_list = (mfs_cfg, '/etc/mfs/mfshdd.cfg', - '/etc/mfshdd.cfg', '/usr/local/etc/mfshdd.cfg') - for conf in mfs_cfg_list: - if not os.path.exists(conf): - continue - f = open(conf) - for line in f: - path = line.strip('#* \n') - if os.path.exists(path): - mfsdirs.append(path) - f.close() -_scan() - -CHUNKHDRSIZE = 1024 * 5 - -def read_chunk_from_local(chunkid, version, size, offset=0): - if offset + size > CHUNKSIZE: - raise ValueError("size too large %s > %s" % - (size, CHUNKSIZE-offset)) - - from dpark.accumulator import LocalReadBytes - name = '%02X/chunk_%016X_%08X.mfs' % (chunkid & 0xFF, chunkid, version) - for d in mfsdirs: - p = os.path.join(d, name) - if os.path.exists(p): - if os.path.getsize(p) < CHUNKHDRSIZE + offset + size: - logger.error('%s is not completed: %d < %d', name, - os.path.getsize(p), CHUNKHDRSIZE + offset + size) - return - #raise ValueError("size too large") - f = open(p) - f.seek(CHUNKHDRSIZE + offset) - while size > 0: - to_read = min(size, 640*1024) - data = f.read(to_read) - if not data: - return - LocalReadBytes.add(len(data)) - yield data - size -= len(data) - f.close() - return - else: - logger.warning("%s was not found", name) - - -def read_chunk(host, port, chunkid, version, size, offset=0): - if offset + size > CHUNKSIZE: - raise ValueError("size too large %s > %s" % - (size, CHUNKSIZE-offset)) - - from dpark.accumulator import RemoteReadBytes - - conn = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - conn.settimeout(10) - conn.connect((host, port)) - - msg = pack(CUTOCS_READ, uint64(chunkid), version, offset, size) - n = conn.send(msg) - while n < len(msg): - if not n: - raise IOError("write failed") - msg = msg[n:] - n = conn.send(msg) - - def recv(n): - d = conn.recv(n) - while len(d) < n: - nd = conn.recv(n-len(d)) - if not nd: - raise IOError("not enough data") - d += nd - return d - - while size > 0: - cmd, l = unpack("II", recv(8)) - - if cmd == CSTOCU_READ_STATUS: - if l != 9: - raise Exception("readblock: READ_STATUS incorrect message size") - cid, code = unpack("QB", recv(l)) - if cid != chunkid: - raise Exception("readblock; READ_STATUS incorrect chunkid") - conn.close() - return - - elif cmd == CSTOCU_READ_DATA: - if l < 20 : - raise Exception("readblock; READ_DATA incorrect message size") - cid, bid, boff, bsize, crc = unpack("QHHII", recv(20)) - if cid != chunkid: - raise Exception("readblock; READ_STATUS incorrect chunkid") - if l != 20 + bsize: - raise Exception("readblock; READ_DATA incorrect message size ") - if bsize == 0 : # FIXME - raise Exception("readblock; empty block") - #yield "" - #continue - if bid != offset >> 16: - raise Exception("readblock; READ_DATA incorrect block number") - if boff != offset & 0xFFFF: - raise Exception("readblock; READ_DATA incorrect block offset") - breq = 65536 - boff - if size < breq: - breq = size - if bsize != breq: - raise Exception("readblock; READ_DATA incorrect block size") - - while breq > 0: - data = conn.recv(breq) - if not data: - #print chunkid, version, offset, size, bsize, breq - raise IOError("unexpected ending: need %d" % breq) - RemoteReadBytes.add(len(data)) - yield data - breq -= len(data) - - offset += bsize - size -= bsize - else: - raise Exception("readblock; unknown message: %s" % cmd) - conn.close() - - -def test(): - d = list(read_chunk('192.168.11.3', 9422, 6544760, 1, 6, 0)) - print len(d), sum(len(s) for s in d) - d = list(read_chunk('192.168.11.3', 9422, 6544936, 1, 46039893, 0)) - print len(d), sum(len(s) for s in d) - -if __name__ == '__main__': - test() diff --git a/dpark/moosefs/master.py b/dpark/moosefs/master.py deleted file mode 100644 index 9ead8860..00000000 --- a/dpark/moosefs/master.py +++ /dev/null @@ -1,356 +0,0 @@ -import os -import socket -import threading -import Queue -import time -import struct -import logging - -from consts import * -from utils import * - -logger = logging.getLogger(__name__) - -# mfsmaster need to been patched with dcache -ENABLE_DCACHE = False - -class StatInfo: - def __init__(self, totalspace, availspace, trashspace, - reservedspace, inodes): - self.totalspace = totalspace - self.availspace = availspace - self.trashspace = trashspace - self.reservedspace = reservedspace - self.inodes = inodes - -class Chunk: - def __init__(self, id, length, version, csdata): - self.id = id - self.length = length - self.version = version - self.addrs = self._parse(csdata) - - def _parse(self, csdata): - return [(socket.inet_ntoa(csdata[i:i+4]), - unpack("H", csdata[i+4:i+6])[0]) - for i in range(len(csdata))[::6]] - - def __repr__(self): - return "" % (self.id, self.version, self.length) - -def try_again(f): - def _(self, *a, **kw): - for i in range(3): - try: - return f(self, *a, **kw) - except IOError, e: - self.close() - logger.warning("mfs master connection: %s", e) - time.sleep(2**i*0.1) - else: - raise - return _ - -def spawn(target, *args, **kw): - t = threading.Thread(target=target, name=target.__name__, args=args, kwargs=kw) - t.daemon = True - t.start() - return t - -class MasterConn: - def __init__(self, host='mfsmaster', port=9421): - self.host = host - self.port = port - self.uid = os.getuid() - self.gid = os.getgid() - self.sessionid = 0 - self.conn = None - self.packetid = 0 - self.fail_count = 0 - self.dcache = {} - self.dstat = {} - - self.lock = threading.RLock() - self.reply = Queue.Queue() - self.is_ready = False - spawn(self.heartbeat) - spawn(self.recv_thread) - - def heartbeat(self): - while True: - try: - self.nop() - except Exception, e: - self.close() - time.sleep(2) - - def connect(self): - if self.conn is not None: - return - - for _ in range(10): - try: - self.conn = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - self.conn.connect((self.host, self.port)) - break - except socket.error, e: - self.conn = None - #self.next_try = time.time() + 1.5 ** self.fail_count - self.fail_count += 1 - time.sleep(1.5 ** self.fail_count) - - if not self.conn: - raise IOError("mfsmaster not availbale") - - regbuf = pack(CUTOMA_FUSE_REGISTER, FUSE_REGISTER_BLOB_NOACL, - self.sessionid, VERSION) - self.send(regbuf) - recv = self.recv(8) - cmd, i = unpack("II", recv) - if cmd != MATOCU_FUSE_REGISTER: - raise Exception("got incorrect answer from mfsmaster %s" % cmd) - - if i not in (1, 4): - raise Exception("got incorrect size from mfsmaster") - - data = self.recv(i) - if i == 1: - code, = unpack("B", data) - if code != 0: - raise Exception("mfsmaster register error: " - + mfs_strerror(code)) - if self.sessionid == 0: - self.sessionid, = unpack("I", data) - - self.is_ready = True - - def close(self): - with self.lock: - if self.conn: - self.conn.close() - self.conn = None - self.dcache.clear() - self.is_ready = False - - def send(self, buf): - with self.lock: - conn = self.conn - if not conn: - raise IOError("not connected") - n = conn.send(buf) - while n < len(buf): - sent = conn.send(buf[n:]) - if not sent: - self.close() - raise IOError("write to master failed") - n += sent - - def nop(self): - with self.lock: - self.connect() - msg = pack(ANTOAN_NOP, 0) - self.send(msg) - - def recv(self, n): - with self.lock: - conn = self.conn - if not conn: - raise IOError("not connected") - r = conn.recv(n) - while len(r) < n: - rr = conn.recv(n - len(r)) - if not rr: - self.close() - raise IOError("unexpected error: need %d" % (n-len(r))) - r += rr - return r - - def recv_cmd(self): - d = self.recv(12) - cmd, size = unpack("II", d) - data = self.recv(size-4) if size > 4 else '' - while cmd in (ANTOAN_NOP, MATOCU_FUSE_NOTIFY_ATTR, MATOCU_FUSE_NOTIFY_DIR): - if cmd == ANTOAN_NOP: - pass - elif cmd == MATOCU_FUSE_NOTIFY_ATTR: - while len(data) >= 43: - parent, inode = unpack("II", data) - attr = data[8:43] - if parent in self.dcache: - cache = self.dcache[parent] - for name in cache: - if cache[name].inode == inode: - cache[name] = attrToFileInfo(inode, attr) - break - data = data[43:] - elif cmd == MATOCU_FUSE_NOTIFY_DIR: - while len(data) >= 4: - inode, = unpack("I", data) - if inode in self.dcache: - del self.dcache[inode] - with self.lock: - self.send(pack(CUTOMA_FUSE_DIR_REMOVED, 0, inode)) - data = data[4:] - d = self.recv(12) - cmd, size = unpack("II", d) - data = self.recv(size-4) if size > 4 else '' - return d, data - - def recv_thread(self): - while True: - with self.lock: - if not self.is_ready: - time.sleep(0.01) - continue - try: - r = self.recv_cmd() - self.reply.put(r) - except IOError, e: - self.reply.put(e) - - @try_again - def sendAndReceive(self, cmd, *args): - #print 'sendAndReceive', cmd, args - self.packetid += 1 - msg = pack(cmd, self.packetid, *args) - with self.lock: - self.connect() - while not self.reply.empty(): - self.reply.get_nowait() - self.send(msg) - r = self.reply.get() - if isinstance(r, Exception): - raise r - h, d = r - rcmd, size, pid = unpack("III", h) - if rcmd != cmd+1 or pid != self.packetid or size <= 4: - self.close() - raise Exception("incorrect answer (%s!=%s, %s!=%s, %d<=4", - rcmd, cmd+1, pid, self.packetid, size) - if len(d) == 1 and ord(d[0]) != 0: - raise Error(ord(d[0])) - return d - - def statfs(self): - ans = self.sendAndReceive(CUTOMA_FUSE_STATFS) - return StatInfo(*unpack("QQQQI", ans)) - -# def access(self, inode, modemask): -# return self.sendAndReceive(CUTOMA_FUSE_ACCESS, inode, -# self.uid, self.gid, uint8(modemask)) -# - def lookup(self, parent, name): - if ENABLE_DCACHE: - cache = self.dcache.get(parent) - if cache is None and self.dstat.get(parent, 0) > 1: - cache = self.getdirplus(parent) - if cache is not None: - return cache.get(name), None - - self.dstat[parent] = self.dstat.get(parent, 0) + 1 - - ans = self.sendAndReceive(CUTOMA_FUSE_LOOKUP, parent, - uint8(len(name)), name, 0, 0) - if len(ans) == 1: - return None, "" - if len(ans) != 39: - raise Exception("bad length") - inode, = unpack("I", ans) - return attrToFileInfo(inode, ans[4:]), None - - def getattr(self, inode): - ans = self.sendAndReceive(CUTOMA_FUSE_GETATTR, inode, - self.uid, self.gid) - return attrToFileInfo(inode, ans) - - def readlink(self, inode): - ans = self.sendAndReceive(CUTOMA_FUSE_READLINK, inode) - length, = unpack("I", ans) - if length+4 != len(ans): - raise Exception("invalid length") - return ans[4:-1] - - def getdir(self, inode): - "return: {name: (inode,type)}" - ans = self.sendAndReceive(CUTOMA_FUSE_GETDIR, inode, - self.uid, self.gid) - p = 0 - names = {} - while p < len(ans): - length, = unpack("B", ans[p:p+1]) - p += 1 - if length + p + 5 > len(ans): - break - name = ans[p:p+length] - p += length - inode, type = unpack("IB", ans) - names[name] = (inode, type) - p += 5 - return names - - def getdirplus(self, inode): - "return {name: FileInfo()}" - if ENABLE_DCACHE: - infos = self.dcache.get(inode) - if infos is not None: - return infos - - flag = GETDIR_FLAG_WITHATTR - if ENABLE_DCACHE: - flag |= GETDIR_FLAG_DIRCACHE - ans = self.sendAndReceive(CUTOMA_FUSE_GETDIR, inode, - self.uid, self.gid, uint8(flag)) - p = 0 - infos = {} - while p < len(ans): - length, = unpack("B", ans[p:p+1]) - p += 1 - name = ans[p:p+length] - p += length - i, = unpack("I", ans[p:p+4]) - attr = ans[p+4:p+39] - infos[name] = attrToFileInfo(i, attr, name) - p += 39 - if ENABLE_DCACHE: - self.dcache[inode] = infos - return infos - - def opencheck(self, inode, flag=1): - ans = self.sendAndReceive(CUTOMA_FUSE_OPEN, inode, - self.uid, self.gid, uint8(flag)) - return ans - - def readchunk(self, inode, index): - ans = self.sendAndReceive(CUTOMA_FUSE_READ_CHUNK, inode, index) - n = len(ans) - if n < 20 or (n-20)%6 != 0: - raise Exception("read chunk: invalid length: %s" % n) - length, id, version = unpack("QQI", ans) - return Chunk(id, length, version, ans[20:]) - - -def test(): - m = MasterConn("mfsmaster") - m.connect() - m.close() - #print m.get_attr(1) - while True: - print m.getdir(1) - print m.getdirplus(1) - time.sleep(60) - info, err = m.lookup(1, "test.csv") - print info, err - #print m.opencheck(info.inode) - chunks = m.readchunk(info.inode, 0) - print chunks, chunks.addrs - - for i in range(1000): - info, err = m.lookup(1, "test.csv") - chunks = m.readchunk(info.inode, 0) - print i,err, chunks - time.sleep(10) - - m.close() - -if __name__ == '__main__': - test() diff --git a/dpark/moosefs/utils.py b/dpark/moosefs/utils.py deleted file mode 100644 index 7363f613..00000000 --- a/dpark/moosefs/utils.py +++ /dev/null @@ -1,66 +0,0 @@ -import struct - -from consts import * - -def uint8(n): - return struct.pack("B", n) - -def uint64(n): - return struct.pack("!Q", n) - -class Error(Exception): - def __init__(self, code): - self.code = code - def __str__(self): - return mfs_strerror(self.code) - -def pack(cmd, *args): - msg = [] - for a in args: - if isinstance(a, (int,long)): - msg.append(struct.pack("!I", a)) - elif isinstance(a, str): - msg.append(a) - else: - raise TypeError(str(type(a))+str(a)) - header = struct.pack("!II", cmd, sum(len(i) for i in msg)) - return header + ''.join(msg) - -def unpack(fmt, buf): - if not fmt.startswith("!"): - fmt = "!" + fmt - return struct.unpack(fmt, buf[:struct.calcsize(fmt)]) - -class FileInfo: - def __init__(self, inode, name, type, mode, uid, gid, - atime, mtime, ctime, nlink, length): - self.inode = inode - self.name = name - self.type = chr(type) - if type == TYPE_DIRECTORY: - mode |= S_IFDIR - elif type == TYPE_SYMLINK: - mode |= S_IFLNK - elif type == TYPE_FILE: - mode |= S_IFREG - self.mode = mode - self.uid = uid - self.gid = gid - self.atime = atime - self.mtime = mtime - self.ctime = ctime - self.nlink = nlink - self.length = length - self.blocks = (length + 511) / 512 - - def __repr__(self): - return ("FileInfo(%s, inode=%d, type=%s, length=%d)" % - (self.name, self.inode, self.type, self.length)) - - def is_symlink(self): - return self.type == TYPE_SYMLINK - -def attrToFileInfo(inode, attrs, name=''): - if len(attrs) != 35: - raise Exception("invalid length of attrs") - return FileInfo(inode, name, *struct.unpack("!BHIIIIIIQ", attrs)) diff --git a/dpark/mutable_dict.py b/dpark/mutable_dict.py index 77a82d01..460d21d7 100644 --- a/dpark/mutable_dict.py +++ b/dpark/mutable_dict.py @@ -1,15 +1,16 @@ -import uuid +from __future__ import absolute_import import os -import cPickle -import urllib +from six.moves import urllib, cPickle import struct import glob -import uuid +import uuid as uuid_pkg from dpark.env import env -from dpark.util import compress, decompress +from dpark.utils import compress, decompress, atomic_file from dpark.tracker import GetValueMessage, AddItemMessage from dpark.dependency import HashPartitioner from collections import OrderedDict +import six + class LRUDict(object): def __init__(self, limit=None): @@ -31,15 +32,18 @@ def put(self, key, value): class ConflictValues(object): - def __init__(self, v=[]): + def __init__(self, v=None): + if v is None: + v = [] self.value = list(v) def __repr__(self): return '' % self.value + class MutableDict(object): - def __init__(self, partition_num , cacheLimit=None): - self.uuid = str(uuid.uuid4()) + def __init__(self, partition_num, cacheLimit=None): + self.uuid = str(uuid_pkg.uuid4()) self.partitioner = HashPartitioner(partition_num) self.data = LRUDict(cacheLimit) self.cacheLimit = cacheLimit @@ -49,7 +53,7 @@ def __init__(self, partition_num , cacheLimit=None): self.is_local = True def __getstate__(self): - return (self.uuid, self.partitioner, self.generation, self.cacheLimit) + return self.uuid, self.partitioner, self.generation, self.cacheLimit def __setstate__(self, v): self.uuid, self.partitioner, self.generation, self.cacheLimit = v @@ -66,10 +70,10 @@ def get(self, key): _key = self._get_key(key) values = self.data.get((_key, key)) if values is None: - for k, v in self._fetch_missing(_key).iteritems(): + for k, v in six.iteritems(self._fetch_missing(_key)): self.data.put((_key, k), v) - - values = self.data.get((_key, key)) + if k == key: + values = v return values[0] if values is not None else None @@ -84,21 +88,23 @@ def _flush(self): return updated_keys = {} - path = self._get_path() + dirname = "mutable_dict" + tmppath = env.workdir.alloc_tmp_dir(dirname) + path = env.workdir.export(tmppath, dirname) uri = env.get('SERVER_URI') server_uri = '%s/%s' % (uri, os.path.basename(path)) - for k,v in self.updated.items(): + for k, v in self.updated.items(): key = self._get_key(k) if key in updated_keys: updated_keys[key][k] = v else: - updated_keys[key] = {k:v} + updated_keys[key] = {k: v} - uid = uuid.uuid4().get_hex() + uid = uuid_pkg.uuid4().get_hex() for key, updated in updated_keys.items(): new = self._fetch_missing(key) - for k,v in updated.items(): + for k, v in updated.items(): if v is None: new.pop(k) else: @@ -110,19 +116,18 @@ def _flush(self): raise RuntimeError('conflict uuid for mutable_dict') url = '%s/%s' % (server_uri, filename) - with open(fn+'.tmp', 'wb+') as f: + with atomic_file(fn) as f: data = compress(cPickle.dumps(new)) - f.write(struct.pack('= 0.66: - if not os.path.exists(path): - try: - os.makedirs(path) - except OSError, e: - pass - - return path - - for d in dirs[1:]: - p = os.path.join(d, 'mutable_dict') - try: - os.makedirs(p) - os.symlink(p, path) - except OSError, e: - pass - - return path - - raise RuntimeError('Cannot find suitable workdir') + self.partitioner.getPartition(key)) _all_mutable_dicts = {} + @classmethod def register(cls, md): uuid = md.uuid @@ -236,4 +211,3 @@ def flush(cls): def merge(cls): for md in cls._all_mutable_dicts.values(): md._merge() - diff --git a/dpark/portable_hash.c b/dpark/portable_hash.c deleted file mode 100644 index 383673c4..00000000 --- a/dpark/portable_hash.c +++ /dev/null @@ -1,3010 +0,0 @@ -/* Generated by Cython 0.19.1 on Tue Oct 29 17:34:55 2013 */ - -#define PY_SSIZE_T_CLEAN -#ifndef CYTHON_USE_PYLONG_INTERNALS -#ifdef PYLONG_BITS_IN_DIGIT -#define CYTHON_USE_PYLONG_INTERNALS 0 -#else -#include "pyconfig.h" -#ifdef PYLONG_BITS_IN_DIGIT -#define CYTHON_USE_PYLONG_INTERNALS 1 -#else -#define CYTHON_USE_PYLONG_INTERNALS 0 -#endif -#endif -#endif -#include "Python.h" -#ifndef Py_PYTHON_H - #error Python headers needed to compile C extensions, please install development version of Python. -#elif PY_VERSION_HEX < 0x02040000 - #error Cython requires Python 2.4+. -#else -#include /* For offsetof */ -#ifndef offsetof -#define offsetof(type, member) ( (size_t) & ((type*)0) -> member ) -#endif -#if !defined(WIN32) && !defined(MS_WINDOWS) - #ifndef __stdcall - #define __stdcall - #endif - #ifndef __cdecl - #define __cdecl - #endif - #ifndef __fastcall - #define __fastcall - #endif -#endif -#ifndef DL_IMPORT - #define DL_IMPORT(t) t -#endif -#ifndef DL_EXPORT - #define DL_EXPORT(t) t -#endif -#ifndef PY_LONG_LONG - #define PY_LONG_LONG LONG_LONG -#endif -#ifndef Py_HUGE_VAL - #define Py_HUGE_VAL HUGE_VAL -#endif -#ifdef PYPY_VERSION -#define CYTHON_COMPILING_IN_PYPY 1 -#define CYTHON_COMPILING_IN_CPYTHON 0 -#else -#define CYTHON_COMPILING_IN_PYPY 0 -#define CYTHON_COMPILING_IN_CPYTHON 1 -#endif -#if PY_VERSION_HEX < 0x02050000 - typedef int Py_ssize_t; - #define PY_SSIZE_T_MAX INT_MAX - #define PY_SSIZE_T_MIN INT_MIN - #define PY_FORMAT_SIZE_T "" - #define CYTHON_FORMAT_SSIZE_T "" - #define PyInt_FromSsize_t(z) PyInt_FromLong(z) - #define PyInt_AsSsize_t(o) __Pyx_PyInt_AsInt(o) - #define PyNumber_Index(o) ((PyNumber_Check(o) && !PyFloat_Check(o)) ? PyNumber_Int(o) : \ - (PyErr_Format(PyExc_TypeError, \ - "expected index value, got %.200s", Py_TYPE(o)->tp_name), \ - (PyObject*)0)) - #define __Pyx_PyIndex_Check(o) (PyNumber_Check(o) && !PyFloat_Check(o) && \ - !PyComplex_Check(o)) - #define PyIndex_Check __Pyx_PyIndex_Check - #define PyErr_WarnEx(category, message, stacklevel) PyErr_Warn(category, message) - #define __PYX_BUILD_PY_SSIZE_T "i" -#else - #define __PYX_BUILD_PY_SSIZE_T "n" - #define CYTHON_FORMAT_SSIZE_T "z" - #define __Pyx_PyIndex_Check PyIndex_Check -#endif -#if PY_VERSION_HEX < 0x02060000 - #define Py_REFCNT(ob) (((PyObject*)(ob))->ob_refcnt) - #define Py_TYPE(ob) (((PyObject*)(ob))->ob_type) - #define Py_SIZE(ob) (((PyVarObject*)(ob))->ob_size) - #define PyVarObject_HEAD_INIT(type, size) \ - PyObject_HEAD_INIT(type) size, - #define PyType_Modified(t) - typedef struct { - void *buf; - PyObject *obj; - Py_ssize_t len; - Py_ssize_t itemsize; - int readonly; - int ndim; - char *format; - Py_ssize_t *shape; - Py_ssize_t *strides; - Py_ssize_t *suboffsets; - void *internal; - } Py_buffer; - #define PyBUF_SIMPLE 0 - #define PyBUF_WRITABLE 0x0001 - #define PyBUF_FORMAT 0x0004 - #define PyBUF_ND 0x0008 - #define PyBUF_STRIDES (0x0010 | PyBUF_ND) - #define PyBUF_C_CONTIGUOUS (0x0020 | PyBUF_STRIDES) - #define PyBUF_F_CONTIGUOUS (0x0040 | PyBUF_STRIDES) - #define PyBUF_ANY_CONTIGUOUS (0x0080 | PyBUF_STRIDES) - #define PyBUF_INDIRECT (0x0100 | PyBUF_STRIDES) - #define PyBUF_RECORDS (PyBUF_STRIDES | PyBUF_FORMAT | PyBUF_WRITABLE) - #define PyBUF_FULL (PyBUF_INDIRECT | PyBUF_FORMAT | PyBUF_WRITABLE) - typedef int (*getbufferproc)(PyObject *, Py_buffer *, int); - typedef void (*releasebufferproc)(PyObject *, Py_buffer *); -#endif -#if PY_MAJOR_VERSION < 3 - #define __Pyx_BUILTIN_MODULE_NAME "__builtin__" - #define __Pyx_PyCode_New(a, k, l, s, f, code, c, n, v, fv, cell, fn, name, fline, lnos) \ - PyCode_New(a, l, s, f, code, c, n, v, fv, cell, fn, name, fline, lnos) -#else - #define __Pyx_BUILTIN_MODULE_NAME "builtins" - #define __Pyx_PyCode_New(a, k, l, s, f, code, c, n, v, fv, cell, fn, name, fline, lnos) \ - PyCode_New(a, k, l, s, f, code, c, n, v, fv, cell, fn, name, fline, lnos) -#endif -#if PY_MAJOR_VERSION < 3 && PY_MINOR_VERSION < 6 - #define PyUnicode_FromString(s) PyUnicode_Decode(s, strlen(s), "UTF-8", "strict") -#endif -#if PY_MAJOR_VERSION >= 3 - #define Py_TPFLAGS_CHECKTYPES 0 - #define Py_TPFLAGS_HAVE_INDEX 0 -#endif -#if (PY_VERSION_HEX < 0x02060000) || (PY_MAJOR_VERSION >= 3) - #define Py_TPFLAGS_HAVE_NEWBUFFER 0 -#endif -#if PY_VERSION_HEX < 0x02060000 - #define Py_TPFLAGS_HAVE_VERSION_TAG 0 -#endif -#if PY_VERSION_HEX > 0x03030000 && defined(PyUnicode_KIND) - #define CYTHON_PEP393_ENABLED 1 - #define __Pyx_PyUnicode_READY(op) (likely(PyUnicode_IS_READY(op)) ? \ - 0 : _PyUnicode_Ready((PyObject *)(op))) - #define __Pyx_PyUnicode_GET_LENGTH(u) PyUnicode_GET_LENGTH(u) - #define __Pyx_PyUnicode_READ_CHAR(u, i) PyUnicode_READ_CHAR(u, i) - #define __Pyx_PyUnicode_READ(k, d, i) PyUnicode_READ(k, d, i) -#else - #define CYTHON_PEP393_ENABLED 0 - #define __Pyx_PyUnicode_READY(op) (0) - #define __Pyx_PyUnicode_GET_LENGTH(u) PyUnicode_GET_SIZE(u) - #define __Pyx_PyUnicode_READ_CHAR(u, i) ((Py_UCS4)(PyUnicode_AS_UNICODE(u)[i])) - #define __Pyx_PyUnicode_READ(k, d, i) ((k=k), (Py_UCS4)(((Py_UNICODE*)d)[i])) -#endif -#if PY_MAJOR_VERSION >= 3 - #define PyBaseString_Type PyUnicode_Type - #define PyStringObject PyUnicodeObject - #define PyString_Type PyUnicode_Type - #define PyString_Check PyUnicode_Check - #define PyString_CheckExact PyUnicode_CheckExact -#endif -#if PY_VERSION_HEX < 0x02060000 - #define PyBytesObject PyStringObject - #define PyBytes_Type PyString_Type - #define PyBytes_Check PyString_Check - #define PyBytes_CheckExact PyString_CheckExact - #define PyBytes_FromString PyString_FromString - #define PyBytes_FromStringAndSize PyString_FromStringAndSize - #define PyBytes_FromFormat PyString_FromFormat - #define PyBytes_DecodeEscape PyString_DecodeEscape - #define PyBytes_AsString PyString_AsString - #define PyBytes_AsStringAndSize PyString_AsStringAndSize - #define PyBytes_Size PyString_Size - #define PyBytes_AS_STRING PyString_AS_STRING - #define PyBytes_GET_SIZE PyString_GET_SIZE - #define PyBytes_Repr PyString_Repr - #define PyBytes_Concat PyString_Concat - #define PyBytes_ConcatAndDel PyString_ConcatAndDel -#endif -#if PY_MAJOR_VERSION >= 3 - #define __Pyx_PyBaseString_Check(obj) PyUnicode_Check(obj) - #define __Pyx_PyBaseString_CheckExact(obj) PyUnicode_CheckExact(obj) -#else - #define __Pyx_PyBaseString_Check(obj) (PyString_CheckExact(obj) || PyUnicode_CheckExact(obj) || \ - PyString_Check(obj) || PyUnicode_Check(obj)) - #define __Pyx_PyBaseString_CheckExact(obj) (Py_TYPE(obj) == &PyBaseString_Type) -#endif -#if PY_VERSION_HEX < 0x02060000 - #define PySet_Check(obj) PyObject_TypeCheck(obj, &PySet_Type) - #define PyFrozenSet_Check(obj) PyObject_TypeCheck(obj, &PyFrozenSet_Type) -#endif -#ifndef PySet_CheckExact - #define PySet_CheckExact(obj) (Py_TYPE(obj) == &PySet_Type) -#endif -#define __Pyx_TypeCheck(obj, type) PyObject_TypeCheck(obj, (PyTypeObject *)type) -#if PY_MAJOR_VERSION >= 3 - #define PyIntObject PyLongObject - #define PyInt_Type PyLong_Type - #define PyInt_Check(op) PyLong_Check(op) - #define PyInt_CheckExact(op) PyLong_CheckExact(op) - #define PyInt_FromString PyLong_FromString - #define PyInt_FromUnicode PyLong_FromUnicode - #define PyInt_FromLong PyLong_FromLong - #define PyInt_FromSize_t PyLong_FromSize_t - #define PyInt_FromSsize_t PyLong_FromSsize_t - #define PyInt_AsLong PyLong_AsLong - #define PyInt_AS_LONG PyLong_AS_LONG - #define PyInt_AsSsize_t PyLong_AsSsize_t - #define PyInt_AsUnsignedLongMask PyLong_AsUnsignedLongMask - #define PyInt_AsUnsignedLongLongMask PyLong_AsUnsignedLongLongMask -#endif -#if PY_MAJOR_VERSION >= 3 - #define PyBoolObject PyLongObject -#endif -#if PY_VERSION_HEX < 0x03020000 - typedef long Py_hash_t; - #define __Pyx_PyInt_FromHash_t PyInt_FromLong - #define __Pyx_PyInt_AsHash_t PyInt_AsLong -#else - #define __Pyx_PyInt_FromHash_t PyInt_FromSsize_t - #define __Pyx_PyInt_AsHash_t PyInt_AsSsize_t -#endif -#if (PY_MAJOR_VERSION < 3) || (PY_VERSION_HEX >= 0x03010300) - #define __Pyx_PySequence_GetSlice(obj, a, b) PySequence_GetSlice(obj, a, b) - #define __Pyx_PySequence_SetSlice(obj, a, b, value) PySequence_SetSlice(obj, a, b, value) - #define __Pyx_PySequence_DelSlice(obj, a, b) PySequence_DelSlice(obj, a, b) -#else - #define __Pyx_PySequence_GetSlice(obj, a, b) (unlikely(!(obj)) ? \ - (PyErr_SetString(PyExc_SystemError, "null argument to internal routine"), (PyObject*)0) : \ - (likely((obj)->ob_type->tp_as_mapping) ? (PySequence_GetSlice(obj, a, b)) : \ - (PyErr_Format(PyExc_TypeError, "'%.200s' object is unsliceable", (obj)->ob_type->tp_name), (PyObject*)0))) - #define __Pyx_PySequence_SetSlice(obj, a, b, value) (unlikely(!(obj)) ? \ - (PyErr_SetString(PyExc_SystemError, "null argument to internal routine"), -1) : \ - (likely((obj)->ob_type->tp_as_mapping) ? (PySequence_SetSlice(obj, a, b, value)) : \ - (PyErr_Format(PyExc_TypeError, "'%.200s' object doesn't support slice assignment", (obj)->ob_type->tp_name), -1))) - #define __Pyx_PySequence_DelSlice(obj, a, b) (unlikely(!(obj)) ? \ - (PyErr_SetString(PyExc_SystemError, "null argument to internal routine"), -1) : \ - (likely((obj)->ob_type->tp_as_mapping) ? (PySequence_DelSlice(obj, a, b)) : \ - (PyErr_Format(PyExc_TypeError, "'%.200s' object doesn't support slice deletion", (obj)->ob_type->tp_name), -1))) -#endif -#if PY_MAJOR_VERSION >= 3 - #define PyMethod_New(func, self, klass) ((self) ? PyMethod_New(func, self) : PyInstanceMethod_New(func)) -#endif -#if PY_VERSION_HEX < 0x02050000 - #define __Pyx_GetAttrString(o,n) PyObject_GetAttrString((o),((char *)(n))) - #define __Pyx_SetAttrString(o,n,a) PyObject_SetAttrString((o),((char *)(n)),(a)) - #define __Pyx_DelAttrString(o,n) PyObject_DelAttrString((o),((char *)(n))) -#else - #define __Pyx_GetAttrString(o,n) PyObject_GetAttrString((o),(n)) - #define __Pyx_SetAttrString(o,n,a) PyObject_SetAttrString((o),(n),(a)) - #define __Pyx_DelAttrString(o,n) PyObject_DelAttrString((o),(n)) -#endif -#if PY_VERSION_HEX < 0x02050000 - #define __Pyx_NAMESTR(n) ((char *)(n)) - #define __Pyx_DOCSTR(n) ((char *)(n)) -#else - #define __Pyx_NAMESTR(n) (n) - #define __Pyx_DOCSTR(n) (n) -#endif -#ifndef CYTHON_INLINE - #if defined(__GNUC__) - #define CYTHON_INLINE __inline__ - #elif defined(_MSC_VER) - #define CYTHON_INLINE __inline - #elif defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L - #define CYTHON_INLINE inline - #else - #define CYTHON_INLINE - #endif -#endif -#ifndef CYTHON_RESTRICT - #if defined(__GNUC__) - #define CYTHON_RESTRICT __restrict__ - #elif defined(_MSC_VER) - #define CYTHON_RESTRICT __restrict - #elif defined (__STDC_VERSION__) && __STDC_VERSION__ >= 199901L - #define CYTHON_RESTRICT restrict - #else - #define CYTHON_RESTRICT - #endif -#endif -#ifdef NAN -#define __PYX_NAN() ((float) NAN) -#else -static CYTHON_INLINE float __PYX_NAN() { - /* Initialize NaN. The sign is irrelevant, an exponent with all bits 1 and - a nonzero mantissa means NaN. If the first bit in the mantissa is 1, it is - a quiet NaN. */ - float value; - memset(&value, 0xFF, sizeof(value)); - return value; -} -#endif - - -#if PY_MAJOR_VERSION >= 3 - #define __Pyx_PyNumber_Divide(x,y) PyNumber_TrueDivide(x,y) - #define __Pyx_PyNumber_InPlaceDivide(x,y) PyNumber_InPlaceTrueDivide(x,y) -#else - #define __Pyx_PyNumber_Divide(x,y) PyNumber_Divide(x,y) - #define __Pyx_PyNumber_InPlaceDivide(x,y) PyNumber_InPlaceDivide(x,y) -#endif - -#ifndef __PYX_EXTERN_C - #ifdef __cplusplus - #define __PYX_EXTERN_C extern "C" - #else - #define __PYX_EXTERN_C extern - #endif -#endif - -#if defined(WIN32) || defined(MS_WINDOWS) -#define _USE_MATH_DEFINES -#endif -#include -#define __PYX_HAVE__dpark__portable_hash -#define __PYX_HAVE_API__dpark__portable_hash -#include "stdint.h" -#ifdef _OPENMP -#include -#endif /* _OPENMP */ - -#ifdef PYREX_WITHOUT_ASSERTIONS -#define CYTHON_WITHOUT_ASSERTIONS -#endif - -#ifndef CYTHON_UNUSED -# if defined(__GNUC__) -# if !(defined(__cplusplus)) || (__GNUC__ > 3 || (__GNUC__ == 3 && __GNUC_MINOR__ >= 4)) -# define CYTHON_UNUSED __attribute__ ((__unused__)) -# else -# define CYTHON_UNUSED -# endif -# elif defined(__ICC) || (defined(__INTEL_COMPILER) && !defined(_MSC_VER)) -# define CYTHON_UNUSED __attribute__ ((__unused__)) -# else -# define CYTHON_UNUSED -# endif -#endif -typedef struct {PyObject **p; char *s; const Py_ssize_t n; const char* encoding; - const char is_unicode; const char is_str; const char intern; } __Pyx_StringTabEntry; /*proto*/ - -#define __PYX_DEFAULT_STRING_ENCODING_IS_ASCII 0 -#define __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT 0 -#define __PYX_DEFAULT_STRING_ENCODING "" -#define __Pyx_PyObject_FromString __Pyx_PyBytes_FromString -#define __Pyx_PyObject_FromStringAndSize __Pyx_PyBytes_FromStringAndSize -static CYTHON_INLINE char* __Pyx_PyObject_AsString(PyObject*); -static CYTHON_INLINE char* __Pyx_PyObject_AsStringAndSize(PyObject*, Py_ssize_t* length); -#define __Pyx_PyBytes_FromString PyBytes_FromString -#define __Pyx_PyBytes_FromStringAndSize PyBytes_FromStringAndSize -static CYTHON_INLINE PyObject* __Pyx_PyUnicode_FromString(char*); -#if PY_MAJOR_VERSION < 3 - #define __Pyx_PyStr_FromString __Pyx_PyBytes_FromString - #define __Pyx_PyStr_FromStringAndSize __Pyx_PyBytes_FromStringAndSize -#else - #define __Pyx_PyStr_FromString __Pyx_PyUnicode_FromString - #define __Pyx_PyStr_FromStringAndSize __Pyx_PyUnicode_FromStringAndSize -#endif -#define __Pyx_PyObject_AsUString(s) ((unsigned char*) __Pyx_PyObject_AsString(s)) -#define __Pyx_PyObject_FromUString(s) __Pyx_PyObject_FromString((char*)s) -#define __Pyx_PyBytes_FromUString(s) __Pyx_PyBytes_FromString((char*)s) -#define __Pyx_PyStr_FromUString(s) __Pyx_PyStr_FromString((char*)s) -#define __Pyx_PyUnicode_FromUString(s) __Pyx_PyUnicode_FromString((char*)s) -#if PY_MAJOR_VERSION < 3 -static CYTHON_INLINE size_t __Pyx_Py_UNICODE_strlen(const Py_UNICODE *u) -{ - const Py_UNICODE *u_end = u; - while (*u_end++) ; - return u_end - u - 1; -} -#else -#define __Pyx_Py_UNICODE_strlen Py_UNICODE_strlen -#endif -#define __Pyx_PyUnicode_FromUnicode(u) PyUnicode_FromUnicode(u, __Pyx_Py_UNICODE_strlen(u)) -#define __Pyx_PyUnicode_FromUnicodeAndLength PyUnicode_FromUnicode -#define __Pyx_PyUnicode_AsUnicode PyUnicode_AsUnicode -#define __Pyx_Owned_Py_None(b) (Py_INCREF(Py_None), Py_None) -#define __Pyx_PyBool_FromLong(b) ((b) ? (Py_INCREF(Py_True), Py_True) : (Py_INCREF(Py_False), Py_False)) -static CYTHON_INLINE int __Pyx_PyObject_IsTrue(PyObject*); -static CYTHON_INLINE PyObject* __Pyx_PyNumber_Int(PyObject* x); -static CYTHON_INLINE Py_ssize_t __Pyx_PyIndex_AsSsize_t(PyObject*); -static CYTHON_INLINE PyObject * __Pyx_PyInt_FromSize_t(size_t); -static CYTHON_INLINE size_t __Pyx_PyInt_AsSize_t(PyObject*); -#if CYTHON_COMPILING_IN_CPYTHON -#define __pyx_PyFloat_AsDouble(x) (PyFloat_CheckExact(x) ? PyFloat_AS_DOUBLE(x) : PyFloat_AsDouble(x)) -#else -#define __pyx_PyFloat_AsDouble(x) PyFloat_AsDouble(x) -#endif -#define __pyx_PyFloat_AsFloat(x) ((float) __pyx_PyFloat_AsDouble(x)) -#if PY_MAJOR_VERSION < 3 && __PYX_DEFAULT_STRING_ENCODING_IS_ASCII -static int __Pyx_sys_getdefaultencoding_not_ascii; -static int __Pyx_init_sys_getdefaultencoding_params() { - PyObject* sys = NULL; - PyObject* default_encoding = NULL; - PyObject* ascii_chars_u = NULL; - PyObject* ascii_chars_b = NULL; - sys = PyImport_ImportModule("sys"); - if (sys == NULL) goto bad; - default_encoding = PyObject_CallMethod(sys, (char*) (const char*) "getdefaultencoding", NULL); - if (default_encoding == NULL) goto bad; - if (strcmp(PyBytes_AsString(default_encoding), "ascii") == 0) { - __Pyx_sys_getdefaultencoding_not_ascii = 0; - } else { - const char* default_encoding_c = PyBytes_AS_STRING(default_encoding); - char ascii_chars[128]; - int c; - for (c = 0; c < 128; c++) { - ascii_chars[c] = c; - } - __Pyx_sys_getdefaultencoding_not_ascii = 1; - ascii_chars_u = PyUnicode_DecodeASCII(ascii_chars, 128, NULL); - if (ascii_chars_u == NULL) goto bad; - ascii_chars_b = PyUnicode_AsEncodedString(ascii_chars_u, default_encoding_c, NULL); - if (ascii_chars_b == NULL || strncmp(ascii_chars, PyBytes_AS_STRING(ascii_chars_b), 128) != 0) { - PyErr_Format( - PyExc_ValueError, - "This module compiled with c_string_encoding=ascii, but default encoding '%s' is not a superset of ascii.", - default_encoding_c); - goto bad; - } - } - Py_XDECREF(sys); - Py_XDECREF(default_encoding); - Py_XDECREF(ascii_chars_u); - Py_XDECREF(ascii_chars_b); - return 0; -bad: - Py_XDECREF(sys); - Py_XDECREF(default_encoding); - Py_XDECREF(ascii_chars_u); - Py_XDECREF(ascii_chars_b); - return -1; -} -#endif -#if __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT && PY_MAJOR_VERSION >= 3 -#define __Pyx_PyUnicode_FromStringAndSize(c_str, size) PyUnicode_DecodeUTF8(c_str, size, NULL) -#else -#define __Pyx_PyUnicode_FromStringAndSize(c_str, size) PyUnicode_Decode(c_str, size, __PYX_DEFAULT_STRING_ENCODING, NULL) -#if __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT -static char* __PYX_DEFAULT_STRING_ENCODING; -static int __Pyx_init_sys_getdefaultencoding_params() { - PyObject* sys = NULL; - PyObject* default_encoding = NULL; - char* default_encoding_c; - sys = PyImport_ImportModule("sys"); - if (sys == NULL) goto bad; - default_encoding = PyObject_CallMethod(sys, (char*) (const char*) "getdefaultencoding", NULL); - if (default_encoding == NULL) goto bad; - default_encoding_c = PyBytes_AS_STRING(default_encoding); - __PYX_DEFAULT_STRING_ENCODING = (char*) malloc(strlen(default_encoding_c)); - strcpy(__PYX_DEFAULT_STRING_ENCODING, default_encoding_c); - Py_DECREF(sys); - Py_DECREF(default_encoding); - return 0; -bad: - Py_XDECREF(sys); - Py_XDECREF(default_encoding); - return -1; -} -#endif -#endif - - -#ifdef __GNUC__ - /* Test for GCC > 2.95 */ - #if __GNUC__ > 2 || (__GNUC__ == 2 && (__GNUC_MINOR__ > 95)) - #define likely(x) __builtin_expect(!!(x), 1) - #define unlikely(x) __builtin_expect(!!(x), 0) - #else /* __GNUC__ > 2 ... */ - #define likely(x) (x) - #define unlikely(x) (x) - #endif /* __GNUC__ > 2 ... */ -#else /* __GNUC__ */ - #define likely(x) (x) - #define unlikely(x) (x) -#endif /* __GNUC__ */ - -static PyObject *__pyx_m; -static PyObject *__pyx_d; -static PyObject *__pyx_b; -static PyObject *__pyx_empty_tuple; -static PyObject *__pyx_empty_bytes; -static int __pyx_lineno; -static int __pyx_clineno = 0; -static const char * __pyx_cfilenm= __FILE__; -static const char *__pyx_filename; - - -static const char *__pyx_f[] = { - "portable_hash.pyx", -}; - -/*--- Type declarations ---*/ -#ifndef CYTHON_REFNANNY - #define CYTHON_REFNANNY 0 -#endif -#if CYTHON_REFNANNY - typedef struct { - void (*INCREF)(void*, PyObject*, int); - void (*DECREF)(void*, PyObject*, int); - void (*GOTREF)(void*, PyObject*, int); - void (*GIVEREF)(void*, PyObject*, int); - void* (*SetupContext)(const char*, int, const char*); - void (*FinishContext)(void**); - } __Pyx_RefNannyAPIStruct; - static __Pyx_RefNannyAPIStruct *__Pyx_RefNanny = NULL; - static __Pyx_RefNannyAPIStruct *__Pyx_RefNannyImportAPI(const char *modname); /*proto*/ - #define __Pyx_RefNannyDeclarations void *__pyx_refnanny = NULL; -#ifdef WITH_THREAD - #define __Pyx_RefNannySetupContext(name, acquire_gil) \ - if (acquire_gil) { \ - PyGILState_STATE __pyx_gilstate_save = PyGILState_Ensure(); \ - __pyx_refnanny = __Pyx_RefNanny->SetupContext((name), __LINE__, __FILE__); \ - PyGILState_Release(__pyx_gilstate_save); \ - } else { \ - __pyx_refnanny = __Pyx_RefNanny->SetupContext((name), __LINE__, __FILE__); \ - } -#else - #define __Pyx_RefNannySetupContext(name, acquire_gil) \ - __pyx_refnanny = __Pyx_RefNanny->SetupContext((name), __LINE__, __FILE__) -#endif - #define __Pyx_RefNannyFinishContext() \ - __Pyx_RefNanny->FinishContext(&__pyx_refnanny) - #define __Pyx_INCREF(r) __Pyx_RefNanny->INCREF(__pyx_refnanny, (PyObject *)(r), __LINE__) - #define __Pyx_DECREF(r) __Pyx_RefNanny->DECREF(__pyx_refnanny, (PyObject *)(r), __LINE__) - #define __Pyx_GOTREF(r) __Pyx_RefNanny->GOTREF(__pyx_refnanny, (PyObject *)(r), __LINE__) - #define __Pyx_GIVEREF(r) __Pyx_RefNanny->GIVEREF(__pyx_refnanny, (PyObject *)(r), __LINE__) - #define __Pyx_XINCREF(r) do { if((r) != NULL) {__Pyx_INCREF(r); }} while(0) - #define __Pyx_XDECREF(r) do { if((r) != NULL) {__Pyx_DECREF(r); }} while(0) - #define __Pyx_XGOTREF(r) do { if((r) != NULL) {__Pyx_GOTREF(r); }} while(0) - #define __Pyx_XGIVEREF(r) do { if((r) != NULL) {__Pyx_GIVEREF(r);}} while(0) -#else - #define __Pyx_RefNannyDeclarations - #define __Pyx_RefNannySetupContext(name, acquire_gil) - #define __Pyx_RefNannyFinishContext() - #define __Pyx_INCREF(r) Py_INCREF(r) - #define __Pyx_DECREF(r) Py_DECREF(r) - #define __Pyx_GOTREF(r) - #define __Pyx_GIVEREF(r) - #define __Pyx_XINCREF(r) Py_XINCREF(r) - #define __Pyx_XDECREF(r) Py_XDECREF(r) - #define __Pyx_XGOTREF(r) - #define __Pyx_XGIVEREF(r) -#endif /* CYTHON_REFNANNY */ -#define __Pyx_CLEAR(r) do { PyObject* tmp = ((PyObject*)(r)); r = NULL; __Pyx_DECREF(tmp);} while(0) -#define __Pyx_XCLEAR(r) do { if((r) != NULL) {PyObject* tmp = ((PyObject*)(r)); r = NULL; __Pyx_DECREF(tmp);}} while(0) - -#if CYTHON_COMPILING_IN_CPYTHON -static CYTHON_INLINE PyObject* __Pyx_PyObject_GetAttrStr(PyObject* obj, PyObject* attr_name) { - PyTypeObject* tp = Py_TYPE(obj); - if (likely(tp->tp_getattro)) - return tp->tp_getattro(obj, attr_name); -#if PY_MAJOR_VERSION < 3 - if (likely(tp->tp_getattr)) - return tp->tp_getattr(obj, PyString_AS_STRING(attr_name)); -#endif - return PyObject_GetAttr(obj, attr_name); -} -#else -#define __Pyx_PyObject_GetAttrStr(o,n) PyObject_GetAttr(o,n) -#endif - -static PyObject *__Pyx_GetBuiltinName(PyObject *name); /*proto*/ - -static CYTHON_INLINE void __Pyx_ErrRestore(PyObject *type, PyObject *value, PyObject *tb); /*proto*/ -static CYTHON_INLINE void __Pyx_ErrFetch(PyObject **type, PyObject **value, PyObject **tb); /*proto*/ - -static void __Pyx_WriteUnraisable(const char *name, int clineno, - int lineno, const char *filename); /*proto*/ - -#define __Pyx_GetItemInt(o, i, size, to_py_func, is_list, wraparound, boundscheck) \ - (((size) <= sizeof(Py_ssize_t)) ? \ - __Pyx_GetItemInt_Fast(o, i, is_list, wraparound, boundscheck) : \ - __Pyx_GetItemInt_Generic(o, to_py_func(i))) -#define __Pyx_GetItemInt_List(o, i, size, to_py_func, is_list, wraparound, boundscheck) \ - (((size) <= sizeof(Py_ssize_t)) ? \ - __Pyx_GetItemInt_List_Fast(o, i, wraparound, boundscheck) : \ - __Pyx_GetItemInt_Generic(o, to_py_func(i))) -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_List_Fast(PyObject *o, Py_ssize_t i, - int wraparound, int boundscheck); -#define __Pyx_GetItemInt_Tuple(o, i, size, to_py_func, is_list, wraparound, boundscheck) \ - (((size) <= sizeof(Py_ssize_t)) ? \ - __Pyx_GetItemInt_Tuple_Fast(o, i, wraparound, boundscheck) : \ - __Pyx_GetItemInt_Generic(o, to_py_func(i))) -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Tuple_Fast(PyObject *o, Py_ssize_t i, - int wraparound, int boundscheck); -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Generic(PyObject *o, PyObject* j); -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Fast(PyObject *o, Py_ssize_t i, - int is_list, int wraparound, int boundscheck); - -#define __Pyx_GetItemInt_Unicode(o, i, size, to_py_func, is_list, wraparound, boundscheck) \ - (((size) <= sizeof(Py_ssize_t)) ? \ - __Pyx_GetItemInt_Unicode_Fast(o, i, wraparound, boundscheck) : \ - __Pyx_GetItemInt_Unicode_Generic(o, to_py_func(i))) -static CYTHON_INLINE Py_UCS4 __Pyx_GetItemInt_Unicode_Fast(PyObject* ustring, Py_ssize_t i, - int wraparound, int boundscheck); -static CYTHON_INLINE Py_UCS4 __Pyx_GetItemInt_Unicode_Generic(PyObject* ustring, PyObject* j); - -static CYTHON_INLINE int __Pyx_init_unicode_iteration( - PyObject* ustring, Py_ssize_t *length, void** data, int *kind); /* proto */ - -static void __Pyx_Raise(PyObject *type, PyObject *value, PyObject *tb, PyObject *cause); /*proto*/ - -static CYTHON_INLINE int64_t __Pyx_PyInt_from_py_int64_t(PyObject *); - -static CYTHON_INLINE PyObject *__Pyx_PyInt_to_py_int64_t(int64_t); - -static CYTHON_INLINE unsigned char __Pyx_PyInt_AsUnsignedChar(PyObject *); - -static CYTHON_INLINE unsigned short __Pyx_PyInt_AsUnsignedShort(PyObject *); - -static CYTHON_INLINE unsigned int __Pyx_PyInt_AsUnsignedInt(PyObject *); - -static CYTHON_INLINE char __Pyx_PyInt_AsChar(PyObject *); - -static CYTHON_INLINE short __Pyx_PyInt_AsShort(PyObject *); - -static CYTHON_INLINE int __Pyx_PyInt_AsInt(PyObject *); - -static CYTHON_INLINE signed char __Pyx_PyInt_AsSignedChar(PyObject *); - -static CYTHON_INLINE signed short __Pyx_PyInt_AsSignedShort(PyObject *); - -static CYTHON_INLINE signed int __Pyx_PyInt_AsSignedInt(PyObject *); - -static CYTHON_INLINE int __Pyx_PyInt_AsLongDouble(PyObject *); - -static CYTHON_INLINE unsigned long __Pyx_PyInt_AsUnsignedLong(PyObject *); - -static CYTHON_INLINE unsigned PY_LONG_LONG __Pyx_PyInt_AsUnsignedLongLong(PyObject *); - -static CYTHON_INLINE long __Pyx_PyInt_AsLong(PyObject *); - -static CYTHON_INLINE PY_LONG_LONG __Pyx_PyInt_AsLongLong(PyObject *); - -static CYTHON_INLINE signed long __Pyx_PyInt_AsSignedLong(PyObject *); - -static CYTHON_INLINE signed PY_LONG_LONG __Pyx_PyInt_AsSignedLongLong(PyObject *); - -static int __Pyx_check_binary_version(void); - -typedef struct { - int code_line; - PyCodeObject* code_object; -} __Pyx_CodeObjectCacheEntry; -struct __Pyx_CodeObjectCache { - int count; - int max_count; - __Pyx_CodeObjectCacheEntry* entries; -}; -static struct __Pyx_CodeObjectCache __pyx_code_cache = {0,0,NULL}; -static int __pyx_bisect_code_objects(__Pyx_CodeObjectCacheEntry* entries, int count, int code_line); -static PyCodeObject *__pyx_find_code_object(int code_line); -static void __pyx_insert_code_object(int code_line, PyCodeObject* code_object); - -static void __Pyx_AddTraceback(const char *funcname, int c_line, - int py_line, const char *filename); /*proto*/ - -static int __Pyx_InitStrings(__Pyx_StringTabEntry *t); /*proto*/ - - -/* Module declarations from 'libc.stdint' */ - -/* Module declarations from 'dpark.portable_hash' */ -static int64_t __pyx_f_5dpark_13portable_hash_tuple_hash(PyObject *); /*proto*/ -static int64_t __pyx_f_5dpark_13portable_hash_string_hash(PyObject *); /*proto*/ -static int64_t __pyx_f_5dpark_13portable_hash_unicode_hash(PyObject *); /*proto*/ -static int64_t __pyx_f_5dpark_13portable_hash_portable_hash(PyObject *, int __pyx_skip_dispatch); /*proto*/ -#define __Pyx_MODULE_NAME "dpark.portable_hash" -int __pyx_module_is_main_dpark__portable_hash = 0; - -/* Implementation of 'dpark.portable_hash' */ -static PyObject *__pyx_builtin_ord; -static PyObject *__pyx_builtin_TypeError; -static PyObject *__pyx_pf_5dpark_13portable_hash_portable_hash(CYTHON_UNUSED PyObject *__pyx_self, PyObject *__pyx_v_obj); /* proto */ -static char __pyx_k_1[] = "%s is unhashable by portable_hash"; -static char __pyx_k__ord[] = "ord"; -static char __pyx_k____main__[] = "__main__"; -static char __pyx_k____test__[] = "__test__"; -static char __pyx_k__TypeError[] = "TypeError"; -static PyObject *__pyx_kp_s_1; -static PyObject *__pyx_n_s__TypeError; -static PyObject *__pyx_n_s____main__; -static PyObject *__pyx_n_s____test__; -static PyObject *__pyx_n_s__ord; - -/* "dpark/portable_hash.pyx":3 - * from libc.stdint cimport int64_t - * - * cdef int64_t tuple_hash(tuple obj): # <<<<<<<<<<<<<< - * cdef int64_t mul = 1000003, l = len(obj), value = 0x345678, v - * for i in obj: - */ - -static int64_t __pyx_f_5dpark_13portable_hash_tuple_hash(PyObject *__pyx_v_obj) { - int64_t __pyx_v_mul; - int64_t __pyx_v_l; - int64_t __pyx_v_value; - int64_t __pyx_v_v; - PyObject *__pyx_v_i = NULL; - int64_t __pyx_r; - __Pyx_RefNannyDeclarations - Py_ssize_t __pyx_t_1; - PyObject *__pyx_t_2 = NULL; - PyObject *__pyx_t_3 = NULL; - int64_t __pyx_t_4; - int __pyx_t_5; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannySetupContext("tuple_hash", 0); - - /* "dpark/portable_hash.pyx":4 - * - * cdef int64_t tuple_hash(tuple obj): - * cdef int64_t mul = 1000003, l = len(obj), value = 0x345678, v # <<<<<<<<<<<<<< - * for i in obj: - * l -= 1 - */ - __pyx_v_mul = 1000003; - if (unlikely(((PyObject *)__pyx_v_obj) == Py_None)) { - PyErr_SetString(PyExc_TypeError, "object of type 'NoneType' has no len()"); - {__pyx_filename = __pyx_f[0]; __pyx_lineno = 4; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - __pyx_t_1 = PyTuple_GET_SIZE(((PyObject *)__pyx_v_obj)); if (unlikely(__pyx_t_1 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 4; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_v_l = __pyx_t_1; - __pyx_v_value = 0x345678; - - /* "dpark/portable_hash.pyx":5 - * cdef int64_t tuple_hash(tuple obj): - * cdef int64_t mul = 1000003, l = len(obj), value = 0x345678, v - * for i in obj: # <<<<<<<<<<<<<< - * l -= 1 - * v = portable_hash(i) - */ - if (unlikely(((PyObject *)__pyx_v_obj) == Py_None)) { - PyErr_SetString(PyExc_TypeError, "'NoneType' object is not iterable"); - {__pyx_filename = __pyx_f[0]; __pyx_lineno = 5; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - __pyx_t_2 = ((PyObject *)__pyx_v_obj); __Pyx_INCREF(__pyx_t_2); __pyx_t_1 = 0; - for (;;) { - if (__pyx_t_1 >= PyTuple_GET_SIZE(__pyx_t_2)) break; - #if CYTHON_COMPILING_IN_CPYTHON - __pyx_t_3 = PyTuple_GET_ITEM(__pyx_t_2, __pyx_t_1); __Pyx_INCREF(__pyx_t_3); __pyx_t_1++; if (unlikely(0 < 0)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 5; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #else - __pyx_t_3 = PySequence_ITEM(__pyx_t_2, __pyx_t_1); __pyx_t_1++; if (unlikely(!__pyx_t_3)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 5; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #endif - __Pyx_XDECREF(__pyx_v_i); - __pyx_v_i = __pyx_t_3; - __pyx_t_3 = 0; - - /* "dpark/portable_hash.pyx":6 - * cdef int64_t mul = 1000003, l = len(obj), value = 0x345678, v - * for i in obj: - * l -= 1 # <<<<<<<<<<<<<< - * v = portable_hash(i) - * if v == -1: - */ - __pyx_v_l = (__pyx_v_l - 1); - - /* "dpark/portable_hash.pyx":7 - * for i in obj: - * l -= 1 - * v = portable_hash(i) # <<<<<<<<<<<<<< - * if v == -1: - * return -1 - */ - __pyx_t_4 = __pyx_f_5dpark_13portable_hash_portable_hash(__pyx_v_i, 0); if (unlikely(__pyx_t_4 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 7; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_v_v = __pyx_t_4; - - /* "dpark/portable_hash.pyx":8 - * l -= 1 - * v = portable_hash(i) - * if v == -1: # <<<<<<<<<<<<<< - * return -1 - * value = (value ^ v) * mul - */ - __pyx_t_5 = ((__pyx_v_v == -1) != 0); - if (__pyx_t_5) { - - /* "dpark/portable_hash.pyx":9 - * v = portable_hash(i) - * if v == -1: - * return -1 # <<<<<<<<<<<<<< - * value = (value ^ v) * mul - * mul += (82520 + l * 2) - */ - __pyx_r = -1; - __Pyx_DECREF(__pyx_t_2); __pyx_t_2 = 0; - goto __pyx_L0; - goto __pyx_L5; - } - __pyx_L5:; - - /* "dpark/portable_hash.pyx":10 - * if v == -1: - * return -1 - * value = (value ^ v) * mul # <<<<<<<<<<<<<< - * mul += (82520 + l * 2) - * value += 97531 - */ - __pyx_v_value = ((__pyx_v_value ^ __pyx_v_v) * __pyx_v_mul); - - /* "dpark/portable_hash.pyx":11 - * return -1 - * value = (value ^ v) * mul - * mul += (82520 + l * 2) # <<<<<<<<<<<<<< - * value += 97531 - * if value == -1: - */ - __pyx_v_mul = (__pyx_v_mul + ((int64_t)(82520 + (__pyx_v_l * 2)))); - } - __Pyx_DECREF(__pyx_t_2); __pyx_t_2 = 0; - - /* "dpark/portable_hash.pyx":12 - * value = (value ^ v) * mul - * mul += (82520 + l * 2) - * value += 97531 # <<<<<<<<<<<<<< - * if value == -1: - * value = -2 - */ - __pyx_v_value = (__pyx_v_value + 97531); - - /* "dpark/portable_hash.pyx":13 - * mul += (82520 + l * 2) - * value += 97531 - * if value == -1: # <<<<<<<<<<<<<< - * value = -2 - * return value - */ - __pyx_t_5 = ((__pyx_v_value == -1) != 0); - if (__pyx_t_5) { - - /* "dpark/portable_hash.pyx":14 - * value += 97531 - * if value == -1: - * value = -2 # <<<<<<<<<<<<<< - * return value - * - */ - __pyx_v_value = -2; - goto __pyx_L6; - } - __pyx_L6:; - - /* "dpark/portable_hash.pyx":15 - * if value == -1: - * value = -2 - * return value # <<<<<<<<<<<<<< - * - * cdef int64_t string_hash(bytes obj): - */ - __pyx_r = __pyx_v_value; - goto __pyx_L0; - - __pyx_r = 0; - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(__pyx_t_2); - __Pyx_XDECREF(__pyx_t_3); - __Pyx_WriteUnraisable("dpark.portable_hash.tuple_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - __pyx_r = 0; - __pyx_L0:; - __Pyx_XDECREF(__pyx_v_i); - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -/* "dpark/portable_hash.pyx":17 - * return value - * - * cdef int64_t string_hash(bytes obj): # <<<<<<<<<<<<<< - * cdef int64_t l = len(obj), value, i = 0, v - * if l == 0: - */ - -static int64_t __pyx_f_5dpark_13portable_hash_string_hash(PyObject *__pyx_v_obj) { - int64_t __pyx_v_l; - int64_t __pyx_v_value; - int64_t __pyx_v_i; - int64_t __pyx_v_v; - int64_t __pyx_r; - __Pyx_RefNannyDeclarations - Py_ssize_t __pyx_t_1; - int __pyx_t_2; - PyObject *__pyx_t_3 = NULL; - PyObject *__pyx_t_4 = NULL; - int64_t __pyx_t_5; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannySetupContext("string_hash", 0); - - /* "dpark/portable_hash.pyx":18 - * - * cdef int64_t string_hash(bytes obj): - * cdef int64_t l = len(obj), value, i = 0, v # <<<<<<<<<<<<<< - * if l == 0: - * return 0 - */ - if (unlikely(((PyObject *)__pyx_v_obj) == Py_None)) { - PyErr_SetString(PyExc_TypeError, "object of type 'NoneType' has no len()"); - {__pyx_filename = __pyx_f[0]; __pyx_lineno = 18; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - __pyx_t_1 = PyBytes_GET_SIZE(((PyObject *)__pyx_v_obj)); if (unlikely(__pyx_t_1 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 18; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_v_l = __pyx_t_1; - __pyx_v_i = 0; - - /* "dpark/portable_hash.pyx":19 - * cdef int64_t string_hash(bytes obj): - * cdef int64_t l = len(obj), value, i = 0, v - * if l == 0: # <<<<<<<<<<<<<< - * return 0 - * v = ord(obj[0]) - */ - __pyx_t_2 = ((__pyx_v_l == 0) != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":20 - * cdef int64_t l = len(obj), value, i = 0, v - * if l == 0: - * return 0 # <<<<<<<<<<<<<< - * v = ord(obj[0]) - * value = v << 7 - */ - __pyx_r = 0; - goto __pyx_L0; - goto __pyx_L3; - } - __pyx_L3:; - - /* "dpark/portable_hash.pyx":21 - * if l == 0: - * return 0 - * v = ord(obj[0]) # <<<<<<<<<<<<<< - * value = v << 7 - * while i < l: - */ - __pyx_t_3 = __Pyx_GetItemInt(((PyObject *)__pyx_v_obj), 0, sizeof(long), PyInt_FromLong, 0, 0, 1); if (!__pyx_t_3) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 21; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_3); - __pyx_t_4 = PyTuple_New(1); if (unlikely(!__pyx_t_4)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 21; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_4); - PyTuple_SET_ITEM(__pyx_t_4, 0, __pyx_t_3); - __Pyx_GIVEREF(__pyx_t_3); - __pyx_t_3 = 0; - __pyx_t_3 = PyObject_Call(__pyx_builtin_ord, ((PyObject *)__pyx_t_4), NULL); if (unlikely(!__pyx_t_3)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 21; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_3); - __Pyx_DECREF(((PyObject *)__pyx_t_4)); __pyx_t_4 = 0; - __pyx_t_5 = __Pyx_PyInt_from_py_int64_t(__pyx_t_3); if (unlikely((__pyx_t_5 == (int64_t)-1) && PyErr_Occurred())) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 21; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_DECREF(__pyx_t_3); __pyx_t_3 = 0; - __pyx_v_v = __pyx_t_5; - - /* "dpark/portable_hash.pyx":22 - * return 0 - * v = ord(obj[0]) - * value = v << 7 # <<<<<<<<<<<<<< - * while i < l: - * v = ord(obj[i]) - */ - __pyx_v_value = (__pyx_v_v << 7); - - /* "dpark/portable_hash.pyx":23 - * v = ord(obj[0]) - * value = v << 7 - * while i < l: # <<<<<<<<<<<<<< - * v = ord(obj[i]) - * value = (1000003 * value) ^ v - */ - while (1) { - __pyx_t_2 = ((__pyx_v_i < __pyx_v_l) != 0); - if (!__pyx_t_2) break; - - /* "dpark/portable_hash.pyx":24 - * value = v << 7 - * while i < l: - * v = ord(obj[i]) # <<<<<<<<<<<<<< - * value = (1000003 * value) ^ v - * i += 1 - */ - __pyx_t_3 = __Pyx_GetItemInt(((PyObject *)__pyx_v_obj), __pyx_v_i, sizeof(int64_t), __Pyx_PyInt_to_py_int64_t, 0, 1, 1); if (!__pyx_t_3) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 24; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_3); - __pyx_t_4 = PyTuple_New(1); if (unlikely(!__pyx_t_4)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 24; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_4); - PyTuple_SET_ITEM(__pyx_t_4, 0, __pyx_t_3); - __Pyx_GIVEREF(__pyx_t_3); - __pyx_t_3 = 0; - __pyx_t_3 = PyObject_Call(__pyx_builtin_ord, ((PyObject *)__pyx_t_4), NULL); if (unlikely(!__pyx_t_3)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 24; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_3); - __Pyx_DECREF(((PyObject *)__pyx_t_4)); __pyx_t_4 = 0; - __pyx_t_5 = __Pyx_PyInt_from_py_int64_t(__pyx_t_3); if (unlikely((__pyx_t_5 == (int64_t)-1) && PyErr_Occurred())) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 24; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_DECREF(__pyx_t_3); __pyx_t_3 = 0; - __pyx_v_v = __pyx_t_5; - - /* "dpark/portable_hash.pyx":25 - * while i < l: - * v = ord(obj[i]) - * value = (1000003 * value) ^ v # <<<<<<<<<<<<<< - * i += 1 - * - */ - __pyx_v_value = ((1000003 * __pyx_v_value) ^ __pyx_v_v); - - /* "dpark/portable_hash.pyx":26 - * v = ord(obj[i]) - * value = (1000003 * value) ^ v - * i += 1 # <<<<<<<<<<<<<< - * - * value ^= l - */ - __pyx_v_i = (__pyx_v_i + 1); - } - - /* "dpark/portable_hash.pyx":28 - * i += 1 - * - * value ^= l # <<<<<<<<<<<<<< - * if value == -1: - * value = -2 - */ - __pyx_v_value = (__pyx_v_value ^ __pyx_v_l); - - /* "dpark/portable_hash.pyx":29 - * - * value ^= l - * if value == -1: # <<<<<<<<<<<<<< - * value = -2 - * return value - */ - __pyx_t_2 = ((__pyx_v_value == -1) != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":30 - * value ^= l - * if value == -1: - * value = -2 # <<<<<<<<<<<<<< - * return value - * - */ - __pyx_v_value = -2; - goto __pyx_L6; - } - __pyx_L6:; - - /* "dpark/portable_hash.pyx":31 - * if value == -1: - * value = -2 - * return value # <<<<<<<<<<<<<< - * - * cdef int64_t unicode_hash(obj): - */ - __pyx_r = __pyx_v_value; - goto __pyx_L0; - - __pyx_r = 0; - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(__pyx_t_3); - __Pyx_XDECREF(__pyx_t_4); - __Pyx_WriteUnraisable("dpark.portable_hash.string_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - __pyx_r = 0; - __pyx_L0:; - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -/* "dpark/portable_hash.pyx":33 - * return value - * - * cdef int64_t unicode_hash(obj): # <<<<<<<<<<<<<< - * cdef unicode s = obj - * cdef int64_t l = len(obj), value, v - */ - -static int64_t __pyx_f_5dpark_13portable_hash_unicode_hash(PyObject *__pyx_v_obj) { - PyObject *__pyx_v_s = 0; - int64_t __pyx_v_l; - int64_t __pyx_v_value; - int64_t __pyx_v_v; - Py_UCS4 __pyx_v_c; - int64_t __pyx_r; - __Pyx_RefNannyDeclarations - Py_ssize_t __pyx_t_1; - int __pyx_t_2; - Py_UCS4 __pyx_t_3; - PyObject *__pyx_t_4 = NULL; - Py_ssize_t __pyx_t_5; - void *__pyx_t_6; - int __pyx_t_7; - int __pyx_t_8; - Py_ssize_t __pyx_t_9; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannySetupContext("unicode_hash", 0); - - /* "dpark/portable_hash.pyx":34 - * - * cdef int64_t unicode_hash(obj): - * cdef unicode s = obj # <<<<<<<<<<<<<< - * cdef int64_t l = len(obj), value, v - * if l == 0: - */ - if (!(likely(PyUnicode_CheckExact(__pyx_v_obj))||((__pyx_v_obj) == Py_None)||(PyErr_Format(PyExc_TypeError, "Expected unicode, got %.200s", Py_TYPE(__pyx_v_obj)->tp_name), 0))) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 34; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_INCREF(__pyx_v_obj); - __pyx_v_s = ((PyObject*)__pyx_v_obj); - - /* "dpark/portable_hash.pyx":35 - * cdef int64_t unicode_hash(obj): - * cdef unicode s = obj - * cdef int64_t l = len(obj), value, v # <<<<<<<<<<<<<< - * if l == 0: - * return 0 - */ - __pyx_t_1 = PyObject_Length(__pyx_v_obj); if (unlikely(__pyx_t_1 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 35; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_v_l = __pyx_t_1; - - /* "dpark/portable_hash.pyx":36 - * cdef unicode s = obj - * cdef int64_t l = len(obj), value, v - * if l == 0: # <<<<<<<<<<<<<< - * return 0 - * v = ord(s[0]) - */ - __pyx_t_2 = ((__pyx_v_l == 0) != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":37 - * cdef int64_t l = len(obj), value, v - * if l == 0: - * return 0 # <<<<<<<<<<<<<< - * v = ord(s[0]) - * value = v << 7 - */ - __pyx_r = 0; - goto __pyx_L0; - goto __pyx_L3; - } - __pyx_L3:; - - /* "dpark/portable_hash.pyx":38 - * if l == 0: - * return 0 - * v = ord(s[0]) # <<<<<<<<<<<<<< - * value = v << 7 - * for c in s: - */ - __pyx_t_3 = __Pyx_GetItemInt_Unicode(((PyObject *)__pyx_v_s), 0, sizeof(long), PyInt_FromLong, 0, 0, 1); if (unlikely(__pyx_t_3 == (Py_UCS4)-1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 38; __pyx_clineno = __LINE__; goto __pyx_L1_error;}; - __pyx_v_v = ((int)__pyx_t_3); - - /* "dpark/portable_hash.pyx":39 - * return 0 - * v = ord(s[0]) - * value = v << 7 # <<<<<<<<<<<<<< - * for c in s: - * v = ord(c) - */ - __pyx_v_value = (__pyx_v_v << 7); - - /* "dpark/portable_hash.pyx":40 - * v = ord(s[0]) - * value = v << 7 - * for c in s: # <<<<<<<<<<<<<< - * v = ord(c) - * value = (1000003 * value) ^ v - */ - if (unlikely(((PyObject *)__pyx_v_s) == Py_None)) { - PyErr_SetString(PyExc_TypeError, "'NoneType' is not iterable"); - {__pyx_filename = __pyx_f[0]; __pyx_lineno = 40; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - __Pyx_INCREF(((PyObject *)__pyx_v_s)); - __pyx_t_4 = __pyx_v_s; - __pyx_t_8 = __Pyx_init_unicode_iteration(((PyObject *)__pyx_t_4), (&__pyx_t_5), (&__pyx_t_6), (&__pyx_t_7)); if (unlikely(__pyx_t_8 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 40; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - for (__pyx_t_9 = 0; __pyx_t_9 < __pyx_t_5; __pyx_t_9++) { - __pyx_t_1 = __pyx_t_9; - __pyx_v_c = __Pyx_PyUnicode_READ(__pyx_t_7, __pyx_t_6, __pyx_t_1); - - /* "dpark/portable_hash.pyx":41 - * value = v << 7 - * for c in s: - * v = ord(c) # <<<<<<<<<<<<<< - * value = (1000003 * value) ^ v - * - */ - __pyx_v_v = ((int)__pyx_v_c); - - /* "dpark/portable_hash.pyx":42 - * for c in s: - * v = ord(c) - * value = (1000003 * value) ^ v # <<<<<<<<<<<<<< - * - * value ^= l - */ - __pyx_v_value = ((1000003 * __pyx_v_value) ^ __pyx_v_v); - } - __Pyx_DECREF(((PyObject *)__pyx_t_4)); __pyx_t_4 = 0; - - /* "dpark/portable_hash.pyx":44 - * value = (1000003 * value) ^ v - * - * value ^= l # <<<<<<<<<<<<<< - * if value == -1: - * value = -2 - */ - __pyx_v_value = (__pyx_v_value ^ __pyx_v_l); - - /* "dpark/portable_hash.pyx":45 - * - * value ^= l - * if value == -1: # <<<<<<<<<<<<<< - * value = -2 - * return value - */ - __pyx_t_2 = ((__pyx_v_value == -1) != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":46 - * value ^= l - * if value == -1: - * value = -2 # <<<<<<<<<<<<<< - * return value - * - */ - __pyx_v_value = -2; - goto __pyx_L6; - } - __pyx_L6:; - - /* "dpark/portable_hash.pyx":47 - * if value == -1: - * value = -2 - * return value # <<<<<<<<<<<<<< - * - * - */ - __pyx_r = __pyx_v_value; - goto __pyx_L0; - - __pyx_r = 0; - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(((PyObject *)__pyx_t_4)); - __Pyx_WriteUnraisable("dpark.portable_hash.unicode_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - __pyx_r = 0; - __pyx_L0:; - __Pyx_XDECREF(__pyx_v_s); - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -/* "dpark/portable_hash.pyx":50 - * - * - * cpdef int64_t portable_hash(obj) except -1: # <<<<<<<<<<<<<< - * t = type(obj) - * if obj is None: - */ - -static PyObject *__pyx_pw_5dpark_13portable_hash_1portable_hash(PyObject *__pyx_self, PyObject *__pyx_v_obj); /*proto*/ -static int64_t __pyx_f_5dpark_13portable_hash_portable_hash(PyObject *__pyx_v_obj, CYTHON_UNUSED int __pyx_skip_dispatch) { - PyObject *__pyx_v_t = NULL; - int64_t __pyx_r; - __Pyx_RefNannyDeclarations - int __pyx_t_1; - int __pyx_t_2; - int __pyx_t_3; - int __pyx_t_4; - Py_hash_t __pyx_t_5; - PyObject *__pyx_t_6 = NULL; - PyObject *__pyx_t_7 = NULL; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannySetupContext("portable_hash", 0); - - /* "dpark/portable_hash.pyx":51 - * - * cpdef int64_t portable_hash(obj) except -1: - * t = type(obj) # <<<<<<<<<<<<<< - * if obj is None: - * return 1315925605 - */ - __Pyx_INCREF(((PyObject *)Py_TYPE(__pyx_v_obj))); - __pyx_v_t = ((PyObject*)((PyObject *)Py_TYPE(__pyx_v_obj))); - - /* "dpark/portable_hash.pyx":52 - * cpdef int64_t portable_hash(obj) except -1: - * t = type(obj) - * if obj is None: # <<<<<<<<<<<<<< - * return 1315925605 - * if t is bytes: - */ - __pyx_t_1 = (__pyx_v_obj == Py_None); - __pyx_t_2 = (__pyx_t_1 != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":53 - * t = type(obj) - * if obj is None: - * return 1315925605 # <<<<<<<<<<<<<< - * if t is bytes: - * return string_hash(obj) - */ - __pyx_r = 1315925605; - goto __pyx_L0; - goto __pyx_L3; - } - __pyx_L3:; - - /* "dpark/portable_hash.pyx":54 - * if obj is None: - * return 1315925605 - * if t is bytes: # <<<<<<<<<<<<<< - * return string_hash(obj) - * elif t is unicode: - */ - __pyx_t_2 = (__pyx_v_t == ((PyObject*)(&PyBytes_Type))); - __pyx_t_1 = (__pyx_t_2 != 0); - if (__pyx_t_1) { - - /* "dpark/portable_hash.pyx":55 - * return 1315925605 - * if t is bytes: - * return string_hash(obj) # <<<<<<<<<<<<<< - * elif t is unicode: - * return unicode_hash(obj) - */ - if (!(likely(PyBytes_CheckExact(__pyx_v_obj))||((__pyx_v_obj) == Py_None)||(PyErr_Format(PyExc_TypeError, "Expected bytes, got %.200s", Py_TYPE(__pyx_v_obj)->tp_name), 0))) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 55; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_r = __pyx_f_5dpark_13portable_hash_string_hash(((PyObject*)__pyx_v_obj)); - goto __pyx_L0; - goto __pyx_L4; - } - - /* "dpark/portable_hash.pyx":56 - * if t is bytes: - * return string_hash(obj) - * elif t is unicode: # <<<<<<<<<<<<<< - * return unicode_hash(obj) - * elif t is tuple: - */ - __pyx_t_1 = (__pyx_v_t == ((PyObject*)(&PyUnicode_Type))); - __pyx_t_2 = (__pyx_t_1 != 0); - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":57 - * return string_hash(obj) - * elif t is unicode: - * return unicode_hash(obj) # <<<<<<<<<<<<<< - * elif t is tuple: - * return tuple_hash(obj) - */ - __pyx_r = __pyx_f_5dpark_13portable_hash_unicode_hash(__pyx_v_obj); - goto __pyx_L0; - goto __pyx_L4; - } - - /* "dpark/portable_hash.pyx":58 - * elif t is unicode: - * return unicode_hash(obj) - * elif t is tuple: # <<<<<<<<<<<<<< - * return tuple_hash(obj) - * elif t is int or t is long or t is float: - */ - __pyx_t_2 = (__pyx_v_t == ((PyObject*)(&PyTuple_Type))); - __pyx_t_1 = (__pyx_t_2 != 0); - if (__pyx_t_1) { - - /* "dpark/portable_hash.pyx":59 - * return unicode_hash(obj) - * elif t is tuple: - * return tuple_hash(obj) # <<<<<<<<<<<<<< - * elif t is int or t is long or t is float: - * return hash(obj) - */ - if (!(likely(PyTuple_CheckExact(__pyx_v_obj))||((__pyx_v_obj) == Py_None)||(PyErr_Format(PyExc_TypeError, "Expected tuple, got %.200s", Py_TYPE(__pyx_v_obj)->tp_name), 0))) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 59; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_r = __pyx_f_5dpark_13portable_hash_tuple_hash(((PyObject*)__pyx_v_obj)); - goto __pyx_L0; - goto __pyx_L4; - } - - /* "dpark/portable_hash.pyx":60 - * elif t is tuple: - * return tuple_hash(obj) - * elif t is int or t is long or t is float: # <<<<<<<<<<<<<< - * return hash(obj) - * else: - */ - __pyx_t_1 = (__pyx_v_t == ((PyObject*)(&PyInt_Type))); - if (!(__pyx_t_1 != 0)) { - __pyx_t_2 = (__pyx_v_t == ((PyObject*)(&PyLong_Type))); - if (!(__pyx_t_2 != 0)) { - __pyx_t_3 = (__pyx_v_t == ((PyObject*)(&PyFloat_Type))); - __pyx_t_4 = (__pyx_t_3 != 0); - } else { - __pyx_t_4 = (__pyx_t_2 != 0); - } - __pyx_t_2 = __pyx_t_4; - } else { - __pyx_t_2 = (__pyx_t_1 != 0); - } - if (__pyx_t_2) { - - /* "dpark/portable_hash.pyx":61 - * return tuple_hash(obj) - * elif t is int or t is long or t is float: - * return hash(obj) # <<<<<<<<<<<<<< - * else: - * raise TypeError('%s is unhashable by portable_hash' % t) - */ - __pyx_t_5 = PyObject_Hash(__pyx_v_obj); if (unlikely(__pyx_t_5 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 61; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_r = __pyx_t_5; - goto __pyx_L0; - goto __pyx_L4; - } - /*else*/ { - - /* "dpark/portable_hash.pyx":63 - * return hash(obj) - * else: - * raise TypeError('%s is unhashable by portable_hash' % t) # <<<<<<<<<<<<<< - */ - __pyx_t_6 = PyNumber_Remainder(((PyObject *)__pyx_kp_s_1), ((PyObject *)__pyx_v_t)); if (unlikely(!__pyx_t_6)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 63; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(((PyObject *)__pyx_t_6)); - __pyx_t_7 = PyTuple_New(1); if (unlikely(!__pyx_t_7)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 63; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_7); - PyTuple_SET_ITEM(__pyx_t_7, 0, ((PyObject *)__pyx_t_6)); - __Pyx_GIVEREF(((PyObject *)__pyx_t_6)); - __pyx_t_6 = 0; - __pyx_t_6 = PyObject_Call(__pyx_builtin_TypeError, ((PyObject *)__pyx_t_7), NULL); if (unlikely(!__pyx_t_6)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 63; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_6); - __Pyx_DECREF(((PyObject *)__pyx_t_7)); __pyx_t_7 = 0; - __Pyx_Raise(__pyx_t_6, 0, 0, 0); - __Pyx_DECREF(__pyx_t_6); __pyx_t_6 = 0; - {__pyx_filename = __pyx_f[0]; __pyx_lineno = 63; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - __pyx_L4:; - - __pyx_r = 0; - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(__pyx_t_6); - __Pyx_XDECREF(__pyx_t_7); - __Pyx_AddTraceback("dpark.portable_hash.portable_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - __pyx_r = -1; - __pyx_L0:; - __Pyx_XDECREF(__pyx_v_t); - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -/* Python wrapper */ -static PyObject *__pyx_pw_5dpark_13portable_hash_1portable_hash(PyObject *__pyx_self, PyObject *__pyx_v_obj); /*proto*/ -static PyObject *__pyx_pw_5dpark_13portable_hash_1portable_hash(PyObject *__pyx_self, PyObject *__pyx_v_obj) { - PyObject *__pyx_r = 0; - __Pyx_RefNannyDeclarations - __Pyx_RefNannySetupContext("portable_hash (wrapper)", 0); - __pyx_r = __pyx_pf_5dpark_13portable_hash_portable_hash(__pyx_self, ((PyObject *)__pyx_v_obj)); - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -/* "dpark/portable_hash.pyx":50 - * - * - * cpdef int64_t portable_hash(obj) except -1: # <<<<<<<<<<<<<< - * t = type(obj) - * if obj is None: - */ - -static PyObject *__pyx_pf_5dpark_13portable_hash_portable_hash(CYTHON_UNUSED PyObject *__pyx_self, PyObject *__pyx_v_obj) { - PyObject *__pyx_r = NULL; - __Pyx_RefNannyDeclarations - int64_t __pyx_t_1; - PyObject *__pyx_t_2 = NULL; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannySetupContext("portable_hash", 0); - __Pyx_XDECREF(__pyx_r); - __pyx_t_1 = __pyx_f_5dpark_13portable_hash_portable_hash(__pyx_v_obj, 0); if (unlikely(__pyx_t_1 == -1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 50; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_t_2 = __Pyx_PyInt_to_py_int64_t(__pyx_t_1); if (unlikely(!__pyx_t_2)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 50; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(__pyx_t_2); - __pyx_r = __pyx_t_2; - __pyx_t_2 = 0; - goto __pyx_L0; - - __pyx_r = Py_None; __Pyx_INCREF(Py_None); - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(__pyx_t_2); - __Pyx_AddTraceback("dpark.portable_hash.portable_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - __pyx_r = NULL; - __pyx_L0:; - __Pyx_XGIVEREF(__pyx_r); - __Pyx_RefNannyFinishContext(); - return __pyx_r; -} - -static PyMethodDef __pyx_methods[] = { - {__Pyx_NAMESTR("portable_hash"), (PyCFunction)__pyx_pw_5dpark_13portable_hash_1portable_hash, METH_O, __Pyx_DOCSTR(0)}, - {0, 0, 0, 0} -}; - -#if PY_MAJOR_VERSION >= 3 -static struct PyModuleDef __pyx_moduledef = { - #if PY_VERSION_HEX < 0x03020000 - { PyObject_HEAD_INIT(NULL) NULL, 0, NULL }, - #else - PyModuleDef_HEAD_INIT, - #endif - __Pyx_NAMESTR("portable_hash"), - 0, /* m_doc */ - -1, /* m_size */ - __pyx_methods /* m_methods */, - NULL, /* m_reload */ - NULL, /* m_traverse */ - NULL, /* m_clear */ - NULL /* m_free */ -}; -#endif - -static __Pyx_StringTabEntry __pyx_string_tab[] = { - {&__pyx_kp_s_1, __pyx_k_1, sizeof(__pyx_k_1), 0, 0, 1, 0}, - {&__pyx_n_s__TypeError, __pyx_k__TypeError, sizeof(__pyx_k__TypeError), 0, 0, 1, 1}, - {&__pyx_n_s____main__, __pyx_k____main__, sizeof(__pyx_k____main__), 0, 0, 1, 1}, - {&__pyx_n_s____test__, __pyx_k____test__, sizeof(__pyx_k____test__), 0, 0, 1, 1}, - {&__pyx_n_s__ord, __pyx_k__ord, sizeof(__pyx_k__ord), 0, 0, 1, 1}, - {0, 0, 0, 0, 0, 0, 0} -}; -static int __Pyx_InitCachedBuiltins(void) { - __pyx_builtin_ord = __Pyx_GetBuiltinName(__pyx_n_s__ord); if (!__pyx_builtin_ord) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 21; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_builtin_TypeError = __Pyx_GetBuiltinName(__pyx_n_s__TypeError); if (!__pyx_builtin_TypeError) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 63; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - return 0; - __pyx_L1_error:; - return -1; -} - -static int __Pyx_InitCachedConstants(void) { - __Pyx_RefNannyDeclarations - __Pyx_RefNannySetupContext("__Pyx_InitCachedConstants", 0); - __Pyx_RefNannyFinishContext(); - return 0; -} - -static int __Pyx_InitGlobals(void) { - if (__Pyx_InitStrings(__pyx_string_tab) < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;}; - return 0; - __pyx_L1_error:; - return -1; -} - -#if PY_MAJOR_VERSION < 3 -PyMODINIT_FUNC initportable_hash(void); /*proto*/ -PyMODINIT_FUNC initportable_hash(void) -#else -PyMODINIT_FUNC PyInit_portable_hash(void); /*proto*/ -PyMODINIT_FUNC PyInit_portable_hash(void) -#endif -{ - PyObject *__pyx_t_1 = NULL; - int __pyx_lineno = 0; - const char *__pyx_filename = NULL; - int __pyx_clineno = 0; - __Pyx_RefNannyDeclarations - #if CYTHON_REFNANNY - __Pyx_RefNanny = __Pyx_RefNannyImportAPI("refnanny"); - if (!__Pyx_RefNanny) { - PyErr_Clear(); - __Pyx_RefNanny = __Pyx_RefNannyImportAPI("Cython.Runtime.refnanny"); - if (!__Pyx_RefNanny) - Py_FatalError("failed to import 'refnanny' module"); - } - #endif - __Pyx_RefNannySetupContext("PyMODINIT_FUNC PyInit_portable_hash(void)", 0); - if ( __Pyx_check_binary_version() < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_empty_tuple = PyTuple_New(0); if (unlikely(!__pyx_empty_tuple)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_empty_bytes = PyBytes_FromStringAndSize("", 0); if (unlikely(!__pyx_empty_bytes)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #ifdef __Pyx_CyFunction_USED - if (__Pyx_CyFunction_init() < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #endif - #ifdef __Pyx_FusedFunction_USED - if (__pyx_FusedFunction_init() < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #endif - #ifdef __Pyx_Generator_USED - if (__pyx_Generator_init() < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #endif - /*--- Library function declarations ---*/ - /*--- Threads initialization code ---*/ - #if defined(__PYX_FORCE_INIT_THREADS) && __PYX_FORCE_INIT_THREADS - #ifdef WITH_THREAD /* Python build with threading support? */ - PyEval_InitThreads(); - #endif - #endif - /*--- Module creation code ---*/ - #if PY_MAJOR_VERSION < 3 - __pyx_m = Py_InitModule4(__Pyx_NAMESTR("portable_hash"), __pyx_methods, 0, 0, PYTHON_API_VERSION); Py_XINCREF(__pyx_m); - #else - __pyx_m = PyModule_Create(&__pyx_moduledef); - #endif - if (unlikely(!__pyx_m)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __pyx_d = PyModule_GetDict(__pyx_m); if (unlikely(!__pyx_d)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - Py_INCREF(__pyx_d); - #if PY_MAJOR_VERSION >= 3 - { - PyObject *modules = PyImport_GetModuleDict(); if (unlikely(!modules)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - if (!PyDict_GetItemString(modules, "dpark.portable_hash")) { - if (unlikely(PyDict_SetItemString(modules, "dpark.portable_hash", __pyx_m) < 0)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - } - } - #endif - __pyx_b = PyImport_AddModule(__Pyx_NAMESTR(__Pyx_BUILTIN_MODULE_NAME)); if (unlikely(!__pyx_b)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #if CYTHON_COMPILING_IN_PYPY - Py_INCREF(__pyx_b); - #endif - if (__Pyx_SetAttrString(__pyx_m, "__builtins__", __pyx_b) < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;}; - /*--- Initialize various global constants etc. ---*/ - if (unlikely(__Pyx_InitGlobals() < 0)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #if PY_MAJOR_VERSION < 3 && (__PYX_DEFAULT_STRING_ENCODING_IS_ASCII || __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT) - if (__Pyx_init_sys_getdefaultencoding_params() < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - #endif - if (__pyx_module_is_main_dpark__portable_hash) { - if (__Pyx_SetAttrString(__pyx_m, "__name__", __pyx_n_s____main__) < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;}; - } - /*--- Builtin init code ---*/ - if (unlikely(__Pyx_InitCachedBuiltins() < 0)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - /*--- Constants init code ---*/ - if (unlikely(__Pyx_InitCachedConstants() < 0)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - /*--- Global init code ---*/ - /*--- Variable export code ---*/ - /*--- Function export code ---*/ - /*--- Type init code ---*/ - /*--- Type import code ---*/ - /*--- Variable import code ---*/ - /*--- Function import code ---*/ - /*--- Execution code ---*/ - - /* "dpark/portable_hash.pyx":1 - * from libc.stdint cimport int64_t # <<<<<<<<<<<<<< - * - * cdef int64_t tuple_hash(tuple obj): - */ - __pyx_t_1 = PyDict_New(); if (unlikely(!__pyx_t_1)) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_GOTREF(((PyObject *)__pyx_t_1)); - if (PyDict_SetItem(__pyx_d, __pyx_n_s____test__, ((PyObject *)__pyx_t_1)) < 0) {__pyx_filename = __pyx_f[0]; __pyx_lineno = 1; __pyx_clineno = __LINE__; goto __pyx_L1_error;} - __Pyx_DECREF(((PyObject *)__pyx_t_1)); __pyx_t_1 = 0; - goto __pyx_L0; - __pyx_L1_error:; - __Pyx_XDECREF(__pyx_t_1); - if (__pyx_m) { - __Pyx_AddTraceback("init dpark.portable_hash", __pyx_clineno, __pyx_lineno, __pyx_filename); - Py_DECREF(__pyx_m); __pyx_m = 0; - } else if (!PyErr_Occurred()) { - PyErr_SetString(PyExc_ImportError, "init dpark.portable_hash"); - } - __pyx_L0:; - __Pyx_RefNannyFinishContext(); - #if PY_MAJOR_VERSION < 3 - return; - #else - return __pyx_m; - #endif -} - -/* Runtime support code */ -#if CYTHON_REFNANNY -static __Pyx_RefNannyAPIStruct *__Pyx_RefNannyImportAPI(const char *modname) { - PyObject *m = NULL, *p = NULL; - void *r = NULL; - m = PyImport_ImportModule((char *)modname); - if (!m) goto end; - p = PyObject_GetAttrString(m, (char *)"RefNannyAPI"); - if (!p) goto end; - r = PyLong_AsVoidPtr(p); -end: - Py_XDECREF(p); - Py_XDECREF(m); - return (__Pyx_RefNannyAPIStruct *)r; -} -#endif /* CYTHON_REFNANNY */ - -static PyObject *__Pyx_GetBuiltinName(PyObject *name) { - PyObject* result = __Pyx_PyObject_GetAttrStr(__pyx_b, name); - if (unlikely(!result)) { - PyErr_Format(PyExc_NameError, -#if PY_MAJOR_VERSION >= 3 - "name '%U' is not defined", name); -#else - "name '%s' is not defined", PyString_AS_STRING(name)); -#endif - } - return result; -} - -static CYTHON_INLINE void __Pyx_ErrRestore(PyObject *type, PyObject *value, PyObject *tb) { -#if CYTHON_COMPILING_IN_CPYTHON - PyObject *tmp_type, *tmp_value, *tmp_tb; - PyThreadState *tstate = PyThreadState_GET(); - tmp_type = tstate->curexc_type; - tmp_value = tstate->curexc_value; - tmp_tb = tstate->curexc_traceback; - tstate->curexc_type = type; - tstate->curexc_value = value; - tstate->curexc_traceback = tb; - Py_XDECREF(tmp_type); - Py_XDECREF(tmp_value); - Py_XDECREF(tmp_tb); -#else - PyErr_Restore(type, value, tb); -#endif -} -static CYTHON_INLINE void __Pyx_ErrFetch(PyObject **type, PyObject **value, PyObject **tb) { -#if CYTHON_COMPILING_IN_CPYTHON - PyThreadState *tstate = PyThreadState_GET(); - *type = tstate->curexc_type; - *value = tstate->curexc_value; - *tb = tstate->curexc_traceback; - tstate->curexc_type = 0; - tstate->curexc_value = 0; - tstate->curexc_traceback = 0; -#else - PyErr_Fetch(type, value, tb); -#endif -} - -static void __Pyx_WriteUnraisable(const char *name, CYTHON_UNUSED int clineno, - CYTHON_UNUSED int lineno, CYTHON_UNUSED const char *filename) { - PyObject *old_exc, *old_val, *old_tb; - PyObject *ctx; - __Pyx_ErrFetch(&old_exc, &old_val, &old_tb); - #if PY_MAJOR_VERSION < 3 - ctx = PyString_FromString(name); - #else - ctx = PyUnicode_FromString(name); - #endif - __Pyx_ErrRestore(old_exc, old_val, old_tb); - if (!ctx) { - PyErr_WriteUnraisable(Py_None); - } else { - PyErr_WriteUnraisable(ctx); - Py_DECREF(ctx); - } -} - -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Generic(PyObject *o, PyObject* j) { - PyObject *r; - if (!j) return NULL; - r = PyObject_GetItem(o, j); - Py_DECREF(j); - return r; -} -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_List_Fast(PyObject *o, Py_ssize_t i, - int wraparound, int boundscheck) { -#if CYTHON_COMPILING_IN_CPYTHON - if (wraparound & unlikely(i < 0)) i += PyList_GET_SIZE(o); - if ((!boundscheck) || likely((0 <= i) & (i < PyList_GET_SIZE(o)))) { - PyObject *r = PyList_GET_ITEM(o, i); - Py_INCREF(r); - return r; - } - return __Pyx_GetItemInt_Generic(o, PyInt_FromSsize_t(i)); -#else - return PySequence_GetItem(o, i); -#endif -} -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Tuple_Fast(PyObject *o, Py_ssize_t i, - int wraparound, int boundscheck) { -#if CYTHON_COMPILING_IN_CPYTHON - if (wraparound & unlikely(i < 0)) i += PyTuple_GET_SIZE(o); - if ((!boundscheck) || likely((0 <= i) & (i < PyTuple_GET_SIZE(o)))) { - PyObject *r = PyTuple_GET_ITEM(o, i); - Py_INCREF(r); - return r; - } - return __Pyx_GetItemInt_Generic(o, PyInt_FromSsize_t(i)); -#else - return PySequence_GetItem(o, i); -#endif -} -static CYTHON_INLINE PyObject *__Pyx_GetItemInt_Fast(PyObject *o, Py_ssize_t i, - int is_list, int wraparound, int boundscheck) { -#if CYTHON_COMPILING_IN_CPYTHON - if (is_list || PyList_CheckExact(o)) { - Py_ssize_t n = ((!wraparound) | likely(i >= 0)) ? i : i + PyList_GET_SIZE(o); - if ((!boundscheck) || (likely((n >= 0) & (n < PyList_GET_SIZE(o))))) { - PyObject *r = PyList_GET_ITEM(o, n); - Py_INCREF(r); - return r; - } - } - else if (PyTuple_CheckExact(o)) { - Py_ssize_t n = ((!wraparound) | likely(i >= 0)) ? i : i + PyTuple_GET_SIZE(o); - if ((!boundscheck) || likely((n >= 0) & (n < PyTuple_GET_SIZE(o)))) { - PyObject *r = PyTuple_GET_ITEM(o, n); - Py_INCREF(r); - return r; - } - } else { - PySequenceMethods *m = Py_TYPE(o)->tp_as_sequence; - if (likely(m && m->sq_item)) { - if (wraparound && unlikely(i < 0) && likely(m->sq_length)) { - Py_ssize_t l = m->sq_length(o); - if (likely(l >= 0)) { - i += l; - } else { - if (PyErr_ExceptionMatches(PyExc_OverflowError)) - PyErr_Clear(); - else - return NULL; - } - } - return m->sq_item(o, i); - } - } -#else - if (is_list || PySequence_Check(o)) { - return PySequence_GetItem(o, i); - } -#endif - return __Pyx_GetItemInt_Generic(o, PyInt_FromSsize_t(i)); -} - -static CYTHON_INLINE Py_UCS4 __Pyx_GetItemInt_Unicode_Fast(PyObject* ustring, Py_ssize_t i, - int wraparound, int boundscheck) { - Py_ssize_t length; -#if CYTHON_PEP393_ENABLED - if (unlikely(__Pyx_PyUnicode_READY(ustring) < 0)) return (Py_UCS4)-1; -#endif - if (wraparound | boundscheck) { - length = __Pyx_PyUnicode_GET_LENGTH(ustring); - if (wraparound & unlikely(i < 0)) i += length; - if ((!boundscheck) || likely((0 <= i) & (i < length))) { - return __Pyx_PyUnicode_READ_CHAR(ustring, i); - } else { - PyErr_SetString(PyExc_IndexError, "string index out of range"); - return (Py_UCS4)-1; - } - } else { - return __Pyx_PyUnicode_READ_CHAR(ustring, i); - } -} -static CYTHON_INLINE Py_UCS4 __Pyx_GetItemInt_Unicode_Generic(PyObject* ustring, PyObject* j) { - Py_UCS4 uchar; - PyObject *uchar_string; - if (!j) return (Py_UCS4)-1; - uchar_string = PyObject_GetItem(ustring, j); - Py_DECREF(j); - if (!uchar_string) return (Py_UCS4)-1; -#if CYTHON_PEP393_ENABLED - if (unlikely(__Pyx_PyUnicode_READY(uchar_string) < 0)) { - Py_DECREF(uchar_string); - return (Py_UCS4)-1; - } -#endif - uchar = __Pyx_PyUnicode_READ_CHAR(uchar_string, 0); - Py_DECREF(uchar_string); - return uchar; -} - -static CYTHON_INLINE int __Pyx_init_unicode_iteration( - PyObject* ustring, Py_ssize_t *length, void** data, int *kind) { -#if CYTHON_PEP393_ENABLED - if (unlikely(__Pyx_PyUnicode_READY(ustring) < 0)) return -1; - *kind = PyUnicode_KIND(ustring); - *length = PyUnicode_GET_LENGTH(ustring); - *data = PyUnicode_DATA(ustring); -#else - *kind = 0; - *length = PyUnicode_GET_SIZE(ustring); - *data = (void*)PyUnicode_AS_UNICODE(ustring); -#endif - return 0; -} - -#if PY_MAJOR_VERSION < 3 -static void __Pyx_Raise(PyObject *type, PyObject *value, PyObject *tb, - CYTHON_UNUSED PyObject *cause) { - Py_XINCREF(type); - if (!value || value == Py_None) - value = NULL; - else - Py_INCREF(value); - if (!tb || tb == Py_None) - tb = NULL; - else { - Py_INCREF(tb); - if (!PyTraceBack_Check(tb)) { - PyErr_SetString(PyExc_TypeError, - "raise: arg 3 must be a traceback or None"); - goto raise_error; - } - } - #if PY_VERSION_HEX < 0x02050000 - if (PyClass_Check(type)) { - #else - if (PyType_Check(type)) { - #endif -#if CYTHON_COMPILING_IN_PYPY - if (!value) { - Py_INCREF(Py_None); - value = Py_None; - } -#endif - PyErr_NormalizeException(&type, &value, &tb); - } else { - if (value) { - PyErr_SetString(PyExc_TypeError, - "instance exception may not have a separate value"); - goto raise_error; - } - value = type; - #if PY_VERSION_HEX < 0x02050000 - if (PyInstance_Check(type)) { - type = (PyObject*) ((PyInstanceObject*)type)->in_class; - Py_INCREF(type); - } else { - type = 0; - PyErr_SetString(PyExc_TypeError, - "raise: exception must be an old-style class or instance"); - goto raise_error; - } - #else - type = (PyObject*) Py_TYPE(type); - Py_INCREF(type); - if (!PyType_IsSubtype((PyTypeObject *)type, (PyTypeObject *)PyExc_BaseException)) { - PyErr_SetString(PyExc_TypeError, - "raise: exception class must be a subclass of BaseException"); - goto raise_error; - } - #endif - } - __Pyx_ErrRestore(type, value, tb); - return; -raise_error: - Py_XDECREF(value); - Py_XDECREF(type); - Py_XDECREF(tb); - return; -} -#else /* Python 3+ */ -static void __Pyx_Raise(PyObject *type, PyObject *value, PyObject *tb, PyObject *cause) { - PyObject* owned_instance = NULL; - if (tb == Py_None) { - tb = 0; - } else if (tb && !PyTraceBack_Check(tb)) { - PyErr_SetString(PyExc_TypeError, - "raise: arg 3 must be a traceback or None"); - goto bad; - } - if (value == Py_None) - value = 0; - if (PyExceptionInstance_Check(type)) { - if (value) { - PyErr_SetString(PyExc_TypeError, - "instance exception may not have a separate value"); - goto bad; - } - value = type; - type = (PyObject*) Py_TYPE(value); - } else if (PyExceptionClass_Check(type)) { - PyObject *args; - if (!value) - args = PyTuple_New(0); - else if (PyTuple_Check(value)) { - Py_INCREF(value); - args = value; - } else - args = PyTuple_Pack(1, value); - if (!args) - goto bad; - owned_instance = PyEval_CallObject(type, args); - Py_DECREF(args); - if (!owned_instance) - goto bad; - value = owned_instance; - if (!PyExceptionInstance_Check(value)) { - PyErr_Format(PyExc_TypeError, - "calling %R should have returned an instance of " - "BaseException, not %R", - type, Py_TYPE(value)); - goto bad; - } - } else { - PyErr_SetString(PyExc_TypeError, - "raise: exception class must be a subclass of BaseException"); - goto bad; - } -#if PY_VERSION_HEX >= 0x03030000 - if (cause) { -#else - if (cause && cause != Py_None) { -#endif - PyObject *fixed_cause; - if (cause == Py_None) { - fixed_cause = NULL; - } else if (PyExceptionClass_Check(cause)) { - fixed_cause = PyObject_CallObject(cause, NULL); - if (fixed_cause == NULL) - goto bad; - } else if (PyExceptionInstance_Check(cause)) { - fixed_cause = cause; - Py_INCREF(fixed_cause); - } else { - PyErr_SetString(PyExc_TypeError, - "exception causes must derive from " - "BaseException"); - goto bad; - } - PyException_SetCause(value, fixed_cause); - } - PyErr_SetObject(type, value); - if (tb) { - PyThreadState *tstate = PyThreadState_GET(); - PyObject* tmp_tb = tstate->curexc_traceback; - if (tb != tmp_tb) { - Py_INCREF(tb); - tstate->curexc_traceback = tb; - Py_XDECREF(tmp_tb); - } - } -bad: - Py_XDECREF(owned_instance); - return; -} -#endif - -static CYTHON_INLINE int64_t __Pyx_PyInt_from_py_int64_t(PyObject* x) { - const int64_t neg_one = (int64_t)-1, const_zero = (int64_t)0; - const int is_unsigned = const_zero < neg_one; - if (sizeof(int64_t) == sizeof(char)) { - if (is_unsigned) - return (int64_t)__Pyx_PyInt_AsUnsignedChar(x); - else - return (int64_t)__Pyx_PyInt_AsSignedChar(x); - } else if (sizeof(int64_t) == sizeof(short)) { - if (is_unsigned) - return (int64_t)__Pyx_PyInt_AsUnsignedShort(x); - else - return (int64_t)__Pyx_PyInt_AsSignedShort(x); - } else if (sizeof(int64_t) == sizeof(int)) { - if (is_unsigned) - return (int64_t)__Pyx_PyInt_AsUnsignedInt(x); - else - return (int64_t)__Pyx_PyInt_AsSignedInt(x); - } else if (sizeof(int64_t) == sizeof(long)) { - if (is_unsigned) - return (int64_t)__Pyx_PyInt_AsUnsignedLong(x); - else - return (int64_t)__Pyx_PyInt_AsSignedLong(x); - } else if (sizeof(int64_t) == sizeof(PY_LONG_LONG)) { - if (is_unsigned) - return (int64_t)__Pyx_PyInt_AsUnsignedLongLong(x); - else - return (int64_t)__Pyx_PyInt_AsSignedLongLong(x); - } else { - #if CYTHON_COMPILING_IN_PYPY && !defined(_PyLong_AsByteArray) - PyErr_SetString(PyExc_RuntimeError, - "_PyLong_AsByteArray() not available in PyPy, cannot convert large numbers"); - #else - int64_t val; - PyObject *v = __Pyx_PyNumber_Int(x); - #if PY_MAJOR_VERSION < 3 - if (likely(v) && !PyLong_Check(v)) { - PyObject *tmp = v; - v = PyNumber_Long(tmp); - Py_DECREF(tmp); - } - #endif - if (likely(v)) { - int one = 1; int is_little = (int)*(unsigned char *)&one; - unsigned char *bytes = (unsigned char *)&val; - int ret = _PyLong_AsByteArray((PyLongObject *)v, - bytes, sizeof(val), - is_little, !is_unsigned); - Py_DECREF(v); - if (likely(!ret)) - return val; - } - #endif - return (int64_t)-1; - } -} - -static CYTHON_INLINE PyObject *__Pyx_PyInt_to_py_int64_t(int64_t val) { - const int64_t neg_one = (int64_t)-1, const_zero = (int64_t)0; - const int is_unsigned = const_zero < neg_one; - if ((sizeof(int64_t) == sizeof(char)) || - (sizeof(int64_t) == sizeof(short))) { - return PyInt_FromLong((long)val); - } else if ((sizeof(int64_t) == sizeof(int)) || - (sizeof(int64_t) == sizeof(long))) { - if (is_unsigned) - return PyLong_FromUnsignedLong((unsigned long)val); - else - return PyInt_FromLong((long)val); - } else if (sizeof(int64_t) == sizeof(PY_LONG_LONG)) { - if (is_unsigned) - return PyLong_FromUnsignedLongLong((unsigned PY_LONG_LONG)val); - else - return PyLong_FromLongLong((PY_LONG_LONG)val); - } else { - int one = 1; int little = (int)*(unsigned char *)&one; - unsigned char *bytes = (unsigned char *)&val; - return _PyLong_FromByteArray(bytes, sizeof(int64_t), - little, !is_unsigned); - } -} - -static CYTHON_INLINE unsigned char __Pyx_PyInt_AsUnsignedChar(PyObject* x) { - const unsigned char neg_one = (unsigned char)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(unsigned char) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(unsigned char)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to unsigned char" : - "value too large to convert to unsigned char"); - } - return (unsigned char)-1; - } - return (unsigned char)val; - } - return (unsigned char)__Pyx_PyInt_AsUnsignedLong(x); -} - -static CYTHON_INLINE unsigned short __Pyx_PyInt_AsUnsignedShort(PyObject* x) { - const unsigned short neg_one = (unsigned short)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(unsigned short) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(unsigned short)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to unsigned short" : - "value too large to convert to unsigned short"); - } - return (unsigned short)-1; - } - return (unsigned short)val; - } - return (unsigned short)__Pyx_PyInt_AsUnsignedLong(x); -} - -static CYTHON_INLINE unsigned int __Pyx_PyInt_AsUnsignedInt(PyObject* x) { - const unsigned int neg_one = (unsigned int)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(unsigned int) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(unsigned int)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to unsigned int" : - "value too large to convert to unsigned int"); - } - return (unsigned int)-1; - } - return (unsigned int)val; - } - return (unsigned int)__Pyx_PyInt_AsUnsignedLong(x); -} - -static CYTHON_INLINE char __Pyx_PyInt_AsChar(PyObject* x) { - const char neg_one = (char)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(char) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(char)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to char" : - "value too large to convert to char"); - } - return (char)-1; - } - return (char)val; - } - return (char)__Pyx_PyInt_AsLong(x); -} - -static CYTHON_INLINE short __Pyx_PyInt_AsShort(PyObject* x) { - const short neg_one = (short)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(short) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(short)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to short" : - "value too large to convert to short"); - } - return (short)-1; - } - return (short)val; - } - return (short)__Pyx_PyInt_AsLong(x); -} - -static CYTHON_INLINE int __Pyx_PyInt_AsInt(PyObject* x) { - const int neg_one = (int)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(int) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(int)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to int" : - "value too large to convert to int"); - } - return (int)-1; - } - return (int)val; - } - return (int)__Pyx_PyInt_AsLong(x); -} - -static CYTHON_INLINE signed char __Pyx_PyInt_AsSignedChar(PyObject* x) { - const signed char neg_one = (signed char)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(signed char) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(signed char)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to signed char" : - "value too large to convert to signed char"); - } - return (signed char)-1; - } - return (signed char)val; - } - return (signed char)__Pyx_PyInt_AsSignedLong(x); -} - -static CYTHON_INLINE signed short __Pyx_PyInt_AsSignedShort(PyObject* x) { - const signed short neg_one = (signed short)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(signed short) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(signed short)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to signed short" : - "value too large to convert to signed short"); - } - return (signed short)-1; - } - return (signed short)val; - } - return (signed short)__Pyx_PyInt_AsSignedLong(x); -} - -static CYTHON_INLINE signed int __Pyx_PyInt_AsSignedInt(PyObject* x) { - const signed int neg_one = (signed int)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(signed int) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(signed int)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to signed int" : - "value too large to convert to signed int"); - } - return (signed int)-1; - } - return (signed int)val; - } - return (signed int)__Pyx_PyInt_AsSignedLong(x); -} - -static CYTHON_INLINE int __Pyx_PyInt_AsLongDouble(PyObject* x) { - const int neg_one = (int)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; - if (sizeof(int) < sizeof(long)) { - long val = __Pyx_PyInt_AsLong(x); - if (unlikely(val != (long)(int)val)) { - if (!unlikely(val == -1 && PyErr_Occurred())) { - PyErr_SetString(PyExc_OverflowError, - (is_unsigned && unlikely(val < 0)) ? - "can't convert negative value to int" : - "value too large to convert to int"); - } - return (int)-1; - } - return (int)val; - } - return (int)__Pyx_PyInt_AsLong(x); -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE unsigned long __Pyx_PyInt_AsUnsignedLong(PyObject* x) { - const unsigned long neg_one = (unsigned long)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to unsigned long"); - return (unsigned long)-1; - } - return (unsigned long)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(unsigned long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (unsigned long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to unsigned long"); - return (unsigned long)-1; - } - return (unsigned long)PyLong_AsUnsignedLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(unsigned long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(unsigned long) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(unsigned long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (unsigned long)PyLong_AsLong(x); - } - } else { - unsigned long val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (unsigned long)-1; - val = __Pyx_PyInt_AsUnsignedLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE unsigned PY_LONG_LONG __Pyx_PyInt_AsUnsignedLongLong(PyObject* x) { - const unsigned PY_LONG_LONG neg_one = (unsigned PY_LONG_LONG)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to unsigned PY_LONG_LONG"); - return (unsigned PY_LONG_LONG)-1; - } - return (unsigned PY_LONG_LONG)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(unsigned PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (unsigned PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to unsigned PY_LONG_LONG"); - return (unsigned PY_LONG_LONG)-1; - } - return (unsigned PY_LONG_LONG)PyLong_AsUnsignedLongLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(unsigned PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(unsigned PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(unsigned PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (unsigned PY_LONG_LONG)PyLong_AsLongLong(x); - } - } else { - unsigned PY_LONG_LONG val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (unsigned PY_LONG_LONG)-1; - val = __Pyx_PyInt_AsUnsignedLongLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE long __Pyx_PyInt_AsLong(PyObject* x) { - const long neg_one = (long)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to long"); - return (long)-1; - } - return (long)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to long"); - return (long)-1; - } - return (long)PyLong_AsUnsignedLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(long) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (long)PyLong_AsLong(x); - } - } else { - long val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (long)-1; - val = __Pyx_PyInt_AsLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE PY_LONG_LONG __Pyx_PyInt_AsLongLong(PyObject* x) { - const PY_LONG_LONG neg_one = (PY_LONG_LONG)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to PY_LONG_LONG"); - return (PY_LONG_LONG)-1; - } - return (PY_LONG_LONG)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to PY_LONG_LONG"); - return (PY_LONG_LONG)-1; - } - return (PY_LONG_LONG)PyLong_AsUnsignedLongLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (PY_LONG_LONG)PyLong_AsLongLong(x); - } - } else { - PY_LONG_LONG val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (PY_LONG_LONG)-1; - val = __Pyx_PyInt_AsLongLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE signed long __Pyx_PyInt_AsSignedLong(PyObject* x) { - const signed long neg_one = (signed long)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to signed long"); - return (signed long)-1; - } - return (signed long)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(signed long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (signed long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to signed long"); - return (signed long)-1; - } - return (signed long)PyLong_AsUnsignedLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(signed long)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(signed long) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(signed long) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (signed long)PyLong_AsLong(x); - } - } else { - signed long val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (signed long)-1; - val = __Pyx_PyInt_AsSignedLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS -#include "longintrepr.h" -#endif -#endif -static CYTHON_INLINE signed PY_LONG_LONG __Pyx_PyInt_AsSignedLongLong(PyObject* x) { - const signed PY_LONG_LONG neg_one = (signed PY_LONG_LONG)-1, const_zero = 0; - const int is_unsigned = neg_one > const_zero; -#if PY_MAJOR_VERSION < 3 - if (likely(PyInt_Check(x))) { - long val = PyInt_AS_LONG(x); - if (is_unsigned && unlikely(val < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to signed PY_LONG_LONG"); - return (signed PY_LONG_LONG)-1; - } - return (signed PY_LONG_LONG)val; - } else -#endif - if (likely(PyLong_Check(x))) { - if (is_unsigned) { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(signed PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return (signed PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - if (unlikely(Py_SIZE(x) < 0)) { - PyErr_SetString(PyExc_OverflowError, - "can't convert negative value to signed PY_LONG_LONG"); - return (signed PY_LONG_LONG)-1; - } - return (signed PY_LONG_LONG)PyLong_AsUnsignedLongLong(x); - } else { -#if CYTHON_COMPILING_IN_CPYTHON && PY_MAJOR_VERSION >= 3 -#if CYTHON_USE_PYLONG_INTERNALS - if (sizeof(digit) <= sizeof(signed PY_LONG_LONG)) { - switch (Py_SIZE(x)) { - case 0: return 0; - case 1: return +(signed PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - case -1: return -(signed PY_LONG_LONG) ((PyLongObject*)x)->ob_digit[0]; - } - } -#endif -#endif - return (signed PY_LONG_LONG)PyLong_AsLongLong(x); - } - } else { - signed PY_LONG_LONG val; - PyObject *tmp = __Pyx_PyNumber_Int(x); - if (!tmp) return (signed PY_LONG_LONG)-1; - val = __Pyx_PyInt_AsSignedLongLong(tmp); - Py_DECREF(tmp); - return val; - } -} - -static int __Pyx_check_binary_version(void) { - char ctversion[4], rtversion[4]; - PyOS_snprintf(ctversion, 4, "%d.%d", PY_MAJOR_VERSION, PY_MINOR_VERSION); - PyOS_snprintf(rtversion, 4, "%s", Py_GetVersion()); - if (ctversion[0] != rtversion[0] || ctversion[2] != rtversion[2]) { - char message[200]; - PyOS_snprintf(message, sizeof(message), - "compiletime version %s of module '%.100s' " - "does not match runtime version %s", - ctversion, __Pyx_MODULE_NAME, rtversion); - #if PY_VERSION_HEX < 0x02050000 - return PyErr_Warn(NULL, message); - #else - return PyErr_WarnEx(NULL, message, 1); - #endif - } - return 0; -} - -static int __pyx_bisect_code_objects(__Pyx_CodeObjectCacheEntry* entries, int count, int code_line) { - int start = 0, mid = 0, end = count - 1; - if (end >= 0 && code_line > entries[end].code_line) { - return count; - } - while (start < end) { - mid = (start + end) / 2; - if (code_line < entries[mid].code_line) { - end = mid; - } else if (code_line > entries[mid].code_line) { - start = mid + 1; - } else { - return mid; - } - } - if (code_line <= entries[mid].code_line) { - return mid; - } else { - return mid + 1; - } -} -static PyCodeObject *__pyx_find_code_object(int code_line) { - PyCodeObject* code_object; - int pos; - if (unlikely(!code_line) || unlikely(!__pyx_code_cache.entries)) { - return NULL; - } - pos = __pyx_bisect_code_objects(__pyx_code_cache.entries, __pyx_code_cache.count, code_line); - if (unlikely(pos >= __pyx_code_cache.count) || unlikely(__pyx_code_cache.entries[pos].code_line != code_line)) { - return NULL; - } - code_object = __pyx_code_cache.entries[pos].code_object; - Py_INCREF(code_object); - return code_object; -} -static void __pyx_insert_code_object(int code_line, PyCodeObject* code_object) { - int pos, i; - __Pyx_CodeObjectCacheEntry* entries = __pyx_code_cache.entries; - if (unlikely(!code_line)) { - return; - } - if (unlikely(!entries)) { - entries = (__Pyx_CodeObjectCacheEntry*)PyMem_Malloc(64*sizeof(__Pyx_CodeObjectCacheEntry)); - if (likely(entries)) { - __pyx_code_cache.entries = entries; - __pyx_code_cache.max_count = 64; - __pyx_code_cache.count = 1; - entries[0].code_line = code_line; - entries[0].code_object = code_object; - Py_INCREF(code_object); - } - return; - } - pos = __pyx_bisect_code_objects(__pyx_code_cache.entries, __pyx_code_cache.count, code_line); - if ((pos < __pyx_code_cache.count) && unlikely(__pyx_code_cache.entries[pos].code_line == code_line)) { - PyCodeObject* tmp = entries[pos].code_object; - entries[pos].code_object = code_object; - Py_DECREF(tmp); - return; - } - if (__pyx_code_cache.count == __pyx_code_cache.max_count) { - int new_max = __pyx_code_cache.max_count + 64; - entries = (__Pyx_CodeObjectCacheEntry*)PyMem_Realloc( - __pyx_code_cache.entries, new_max*sizeof(__Pyx_CodeObjectCacheEntry)); - if (unlikely(!entries)) { - return; - } - __pyx_code_cache.entries = entries; - __pyx_code_cache.max_count = new_max; - } - for (i=__pyx_code_cache.count; i>pos; i--) { - entries[i] = entries[i-1]; - } - entries[pos].code_line = code_line; - entries[pos].code_object = code_object; - __pyx_code_cache.count++; - Py_INCREF(code_object); -} - -#include "compile.h" -#include "frameobject.h" -#include "traceback.h" -static PyCodeObject* __Pyx_CreateCodeObjectForTraceback( - const char *funcname, int c_line, - int py_line, const char *filename) { - PyCodeObject *py_code = 0; - PyObject *py_srcfile = 0; - PyObject *py_funcname = 0; - #if PY_MAJOR_VERSION < 3 - py_srcfile = PyString_FromString(filename); - #else - py_srcfile = PyUnicode_FromString(filename); - #endif - if (!py_srcfile) goto bad; - if (c_line) { - #if PY_MAJOR_VERSION < 3 - py_funcname = PyString_FromFormat( "%s (%s:%d)", funcname, __pyx_cfilenm, c_line); - #else - py_funcname = PyUnicode_FromFormat( "%s (%s:%d)", funcname, __pyx_cfilenm, c_line); - #endif - } - else { - #if PY_MAJOR_VERSION < 3 - py_funcname = PyString_FromString(funcname); - #else - py_funcname = PyUnicode_FromString(funcname); - #endif - } - if (!py_funcname) goto bad; - py_code = __Pyx_PyCode_New( - 0, /*int argcount,*/ - 0, /*int kwonlyargcount,*/ - 0, /*int nlocals,*/ - 0, /*int stacksize,*/ - 0, /*int flags,*/ - __pyx_empty_bytes, /*PyObject *code,*/ - __pyx_empty_tuple, /*PyObject *consts,*/ - __pyx_empty_tuple, /*PyObject *names,*/ - __pyx_empty_tuple, /*PyObject *varnames,*/ - __pyx_empty_tuple, /*PyObject *freevars,*/ - __pyx_empty_tuple, /*PyObject *cellvars,*/ - py_srcfile, /*PyObject *filename,*/ - py_funcname, /*PyObject *name,*/ - py_line, /*int firstlineno,*/ - __pyx_empty_bytes /*PyObject *lnotab*/ - ); - Py_DECREF(py_srcfile); - Py_DECREF(py_funcname); - return py_code; -bad: - Py_XDECREF(py_srcfile); - Py_XDECREF(py_funcname); - return NULL; -} -static void __Pyx_AddTraceback(const char *funcname, int c_line, - int py_line, const char *filename) { - PyCodeObject *py_code = 0; - PyObject *py_globals = 0; - PyFrameObject *py_frame = 0; - py_code = __pyx_find_code_object(c_line ? c_line : py_line); - if (!py_code) { - py_code = __Pyx_CreateCodeObjectForTraceback( - funcname, c_line, py_line, filename); - if (!py_code) goto bad; - __pyx_insert_code_object(c_line ? c_line : py_line, py_code); - } - py_globals = PyModule_GetDict(__pyx_m); - if (!py_globals) goto bad; - py_frame = PyFrame_New( - PyThreadState_GET(), /*PyThreadState *tstate,*/ - py_code, /*PyCodeObject *code,*/ - py_globals, /*PyObject *globals,*/ - 0 /*PyObject *locals*/ - ); - if (!py_frame) goto bad; - py_frame->f_lineno = py_line; - PyTraceBack_Here(py_frame); -bad: - Py_XDECREF(py_code); - Py_XDECREF(py_frame); -} - -static int __Pyx_InitStrings(__Pyx_StringTabEntry *t) { - while (t->p) { - #if PY_MAJOR_VERSION < 3 - if (t->is_unicode) { - *t->p = PyUnicode_DecodeUTF8(t->s, t->n - 1, NULL); - } else if (t->intern) { - *t->p = PyString_InternFromString(t->s); - } else { - *t->p = PyString_FromStringAndSize(t->s, t->n - 1); - } - #else /* Python 3+ has unicode identifiers */ - if (t->is_unicode | t->is_str) { - if (t->intern) { - *t->p = PyUnicode_InternFromString(t->s); - } else if (t->encoding) { - *t->p = PyUnicode_Decode(t->s, t->n - 1, t->encoding, NULL); - } else { - *t->p = PyUnicode_FromStringAndSize(t->s, t->n - 1); - } - } else { - *t->p = PyBytes_FromStringAndSize(t->s, t->n - 1); - } - #endif - if (!*t->p) - return -1; - ++t; - } - return 0; -} - -static CYTHON_INLINE PyObject* __Pyx_PyUnicode_FromString(char* c_str) { - return __Pyx_PyUnicode_FromStringAndSize(c_str, strlen(c_str)); -} -static CYTHON_INLINE char* __Pyx_PyObject_AsString(PyObject* o) { - Py_ssize_t ignore; - return __Pyx_PyObject_AsStringAndSize(o, &ignore); -} -static CYTHON_INLINE char* __Pyx_PyObject_AsStringAndSize(PyObject* o, Py_ssize_t *length) { -#if __PYX_DEFAULT_STRING_ENCODING_IS_ASCII || __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT - if ( -#if PY_MAJOR_VERSION < 3 && __PYX_DEFAULT_STRING_ENCODING_IS_ASCII - __Pyx_sys_getdefaultencoding_not_ascii && -#endif - PyUnicode_Check(o)) { -#if PY_VERSION_HEX < 0x03030000 - char* defenc_c; - PyObject* defenc = _PyUnicode_AsDefaultEncodedString(o, NULL); - if (!defenc) return NULL; - defenc_c = PyBytes_AS_STRING(defenc); -#if __PYX_DEFAULT_STRING_ENCODING_IS_ASCII - { - char* end = defenc_c + PyBytes_GET_SIZE(defenc); - char* c; - for (c = defenc_c; c < end; c++) { - if ((unsigned char) (*c) >= 128) { - PyUnicode_AsASCIIString(o); - return NULL; - } - } - } -#endif /*__PYX_DEFAULT_STRING_ENCODING_IS_ASCII*/ - *length = PyBytes_GET_SIZE(defenc); - return defenc_c; -#else /* PY_VERSION_HEX < 0x03030000 */ - if (PyUnicode_READY(o) == -1) return NULL; -#if __PYX_DEFAULT_STRING_ENCODING_IS_ASCII - if (PyUnicode_IS_ASCII(o)) { - *length = PyUnicode_GET_DATA_SIZE(o); - return PyUnicode_AsUTF8(o); - } else { - PyUnicode_AsASCIIString(o); - return NULL; - } -#else /* __PYX_DEFAULT_STRING_ENCODING_IS_ASCII */ - return PyUnicode_AsUTF8AndSize(o, length); -#endif /* __PYX_DEFAULT_STRING_ENCODING_IS_ASCII */ -#endif /* PY_VERSION_HEX < 0x03030000 */ - } else -#endif /* __PYX_DEFAULT_STRING_ENCODING_IS_ASCII || __PYX_DEFAULT_STRING_ENCODING_IS_DEFAULT */ - { - char* result; - int r = PyBytes_AsStringAndSize(o, &result, length); - if (r < 0) { - return NULL; - } else { - return result; - } - } -} -static CYTHON_INLINE int __Pyx_PyObject_IsTrue(PyObject* x) { - int is_true = x == Py_True; - if (is_true | (x == Py_False) | (x == Py_None)) return is_true; - else return PyObject_IsTrue(x); -} -static CYTHON_INLINE PyObject* __Pyx_PyNumber_Int(PyObject* x) { - PyNumberMethods *m; - const char *name = NULL; - PyObject *res = NULL; -#if PY_MAJOR_VERSION < 3 - if (PyInt_Check(x) || PyLong_Check(x)) -#else - if (PyLong_Check(x)) -#endif - return Py_INCREF(x), x; - m = Py_TYPE(x)->tp_as_number; -#if PY_MAJOR_VERSION < 3 - if (m && m->nb_int) { - name = "int"; - res = PyNumber_Int(x); - } - else if (m && m->nb_long) { - name = "long"; - res = PyNumber_Long(x); - } -#else - if (m && m->nb_int) { - name = "int"; - res = PyNumber_Long(x); - } -#endif - if (res) { -#if PY_MAJOR_VERSION < 3 - if (!PyInt_Check(res) && !PyLong_Check(res)) { -#else - if (!PyLong_Check(res)) { -#endif - PyErr_Format(PyExc_TypeError, - "__%s__ returned non-%s (type %.200s)", - name, name, Py_TYPE(res)->tp_name); - Py_DECREF(res); - return NULL; - } - } - else if (!PyErr_Occurred()) { - PyErr_SetString(PyExc_TypeError, - "an integer is required"); - } - return res; -} -static CYTHON_INLINE Py_ssize_t __Pyx_PyIndex_AsSsize_t(PyObject* b) { - Py_ssize_t ival; - PyObject* x = PyNumber_Index(b); - if (!x) return -1; - ival = PyInt_AsSsize_t(x); - Py_DECREF(x); - return ival; -} -static CYTHON_INLINE PyObject * __Pyx_PyInt_FromSize_t(size_t ival) { -#if PY_VERSION_HEX < 0x02050000 - if (ival <= LONG_MAX) - return PyInt_FromLong((long)ival); - else { - unsigned char *bytes = (unsigned char *) &ival; - int one = 1; int little = (int)*(unsigned char*)&one; - return _PyLong_FromByteArray(bytes, sizeof(size_t), little, 0); - } -#else - return PyInt_FromSize_t(ival); -#endif -} -static CYTHON_INLINE size_t __Pyx_PyInt_AsSize_t(PyObject* x) { - unsigned PY_LONG_LONG val = __Pyx_PyInt_AsUnsignedLongLong(x); - if (unlikely(val != (unsigned PY_LONG_LONG)(size_t)val)) { - if ((val != (unsigned PY_LONG_LONG)-1) || !PyErr_Occurred()) - PyErr_SetString(PyExc_OverflowError, - "value too large to convert to size_t"); - return (size_t)-1; - } - return (size_t)val; -} - - -#endif /* Py_PYTHON_H */ diff --git a/dpark/portable_hash.pyx b/dpark/portable_hash.pyx index 76fcb85e..7a84d91b 100644 --- a/dpark/portable_hash.pyx +++ b/dpark/portable_hash.pyx @@ -14,14 +14,15 @@ cdef int64_t tuple_hash(tuple obj): value = -2 return value -cdef int64_t string_hash(bytes obj): - cdef int64_t l = len(obj), value, i = 0, v +cdef int64_t string_hash(bytes obj_): + cdef int64_t l = len(obj_), value, i = 0, v + cdef char* obj = obj_ if l == 0: return 0 - v = ord(obj[0]) + v = obj[0] value = v << 7 while i < l: - v = ord(obj[i]) + v = obj[i] value = (1000003 * value) ^ v i += 1 @@ -60,4 +61,10 @@ cpdef int64_t portable_hash(obj) except -1: elif t is int or t is long or t is float: return hash(obj) else: + try: + import numpy as np + if isinstance(obj, np.number): + return hash(obj) + except ImportError: + pass raise TypeError('%s is unhashable by portable_hash' % t) diff --git a/dpark/pymesos/__init__.py b/dpark/pymesos/__init__.py deleted file mode 100644 index 8bb4a260..00000000 --- a/dpark/pymesos/__init__.py +++ /dev/null @@ -1,2 +0,0 @@ -from .scheduler import Scheduler, MesosSchedulerDriver -from .executor import Executor, MesosExecutorDriver diff --git a/dpark/pymesos/detector.py b/dpark/pymesos/detector.py deleted file mode 100644 index 159d5d81..00000000 --- a/dpark/pymesos/detector.py +++ /dev/null @@ -1,65 +0,0 @@ -try: - from zookeeper import ZooKeeperException as ZookeeperError - from zkpython import ZKClient, ChildrenWatch, DataWatch - def adjust_zk_logging_level(): - pass -except ImportError: - from kazoo.client import KazooClient as ZKClient - from kazoo.recipe.watchers import ChildrenWatch, DataWatch - from kazoo.exceptions import ZookeeperError - def adjust_zk_logging_level(): - import logging - import kazoo - kazoo.client.log.setLevel(logging.WARNING) - kazoo.protocol.connection.log.setLevel(logging.WARNING) - -class MasterDetector(object): - def __init__(self, uri, agent): - self.uri = uri - self.agent = agent - self.zk = ZKClient(uri, 10) - self.masterSeq = None - - def choose(self, children): - if not children: - self.agent.onNoMasterDetectedMessage() - return True - masterSeq = min(children) - if masterSeq == self.masterSeq: - return True - self.masterSeq = masterSeq - DataWatch(self.zk, '/' + masterSeq, self.notify) - return True - - def notify(self, master_addr, _): - self.agent.onNewMasterDetectedMessage(master_addr) - return False - - def start(self): - adjust_zk_logging_level() - self.zk.start() - try: - ChildrenWatch(self.zk, '', self.choose) - except ZookeeperError: - self.agent.onNoMasterDetectedMessage() - self.stop() - - def stop(self): - try: self.zk.stop() - except: pass - - -def test(): - import logging - logging.basicConfig() - class Agent: - def onNewMasterDetectedMessage(self, addr): - print 'got', addr - def onNoMasterDetectedMessage(self): - print 'no master' - d = MasterDetector('zk1:2181/mesos_master2', Agent()) - d.start() - raw_input("press any key to exit:\n") - -if __name__ == '__main__': - test() diff --git a/dpark/pymesos/executor.py b/dpark/pymesos/executor.py deleted file mode 100644 index 728c69f2..00000000 --- a/dpark/pymesos/executor.py +++ /dev/null @@ -1,96 +0,0 @@ -import os, sys -import time - -from process import UPID, Process, async - -from mesos_pb2 import FrameworkID, ExecutorID -from messages_pb2 import RegisterExecutorMessage, ExecutorToFrameworkMessage, StatusUpdateMessage - -class Executor(object): - #def disconnected(self, driver): pass - #def error(self, driver, message): pass - def registered(self, driver, executrInfo, frameworkInfo, slaveInfo): pass - def launchTask(self, driver, task): pass - def killTask(self, driver, taskId): pass - def frameworkMessage(self, driver, message): pass - def shutdown(self, driver): pass - - -class ExecutorDriver(object): - def start(self): pass - def join(self): pass - def run(self): pass - def abort(self): pass - def stop(self): pass - def sendStatusUpdate(self, update): pass - def sendFrameworkMessage(self, data): pass - -class MesosExecutorDriver(Process, ExecutorDriver): - def __init__(self, executor): - Process.__init__(self, 'executor') - self.executor = executor - - env = os.environ - self.local = bool(env.get('MESOS_LOCAL')) - slave_pid = env.get('MESOS_SLAVE_PID') - assert slave_pid, 'expecting MESOS_SLAVE_PID in environment' - self.slave = UPID(slave_pid) - self.framework_id = FrameworkID() - self.framework_id.value = env.get('MESOS_FRAMEWORK_ID') - self.executor_id = ExecutorID() - self.executor_id.value = env.get('MESOS_EXECUTOR_ID') - self.workDirectory = env.get('MESOS_DIRECTORY') - - def onExecutorRegisteredMessage(self, executor_info, framework_id, - framework_info, slave_id, slave_info): - assert framework_id == self.framework_id - self.slave_id = slave_id - return self.executor.registered(self, executor_info, framework_info, - slave_info) - - def onRunTaskMessage(self, framework_id, framework, pid, task): - return self.executor.launchTask(self, task) - - def onKillTaskMessage(self, framework_id, task_id): - return self.executor.killTask(self, task_id) - - def onFrameworkToExecutorMessage(self, slave_id, framework_id, - executor_id, data): - return self.executor.frameworkMessage(self, data) - - def onShutdownExecutorMessage(self): - self.executor.shutdown(self) - if not self.local: - sys.exit(0) - else: - self.stop() - - def onStatusUpdateAcknowledgementMessage(self, slave_id, framework_id, task_id, uuid): - pass - - def start(self): - Process.start(self) - msg = RegisterExecutorMessage() - msg.framework_id.MergeFrom(self.framework_id) - msg.executor_id.MergeFrom(self.executor_id) - return self.send(self.slave, msg) - - @async - def sendFrameworkMessage(self, data): - msg = ExecutorToFrameworkMessage() - msg.slave_id.MergeFrom(self.slave_id) - msg.framework_id.MergeFrom(self.framework_id) - msg.executor_id.MergeFrom(self.executor_id) - msg.data = data - return self.send(self.slave, msg) - - @async - def sendStatusUpdate(self, status): - msg = StatusUpdateMessage() - msg.update.framework_id.MergeFrom(self.framework_id) - msg.update.executor_id.MergeFrom(self.executor_id) - msg.update.slave_id.MergeFrom(self.slave_id) - msg.update.status.MergeFrom(status) - msg.update.timestamp = time.time() - msg.update.uuid = os.urandom(16) - return self.send(self.slave, msg) diff --git a/dpark/pymesos/launcher.py b/dpark/pymesos/launcher.py deleted file mode 100644 index 18b77a91..00000000 --- a/dpark/pymesos/launcher.py +++ /dev/null @@ -1,78 +0,0 @@ -import os, sys -import subprocess -import pwd -import logging - -logger = logging.getLogger(__name__) - -class Launcher(object): - """ - This class sets up the environment for an executor and then exec()'s it. - It can either be used after a fork() in the slave process, or run as a - standalone program (with the main function in launcher_main.cpp). - - The environment is initialized through for steps: - 1) A work directory for the framework is created by createWorkingDirectory(). - 2) The executor is fetched off HDFS if necessary by fetchExecutor(). - 3) Environment variables are set by setupEnvironment(). - 4) We switch to the framework's user in switchUser(). - - Isolation modules that wish to override the default behaviour can subclass - Launcher and override some of the methods to perform extra actions. - """ - def __init__(self, framework_id, executor_id, commandInfo, user=None, workDirectory='/tmp', - slavepid=None, redirectIO=False, switch_user=False): - self.__dict__.update(locals()) - - def run(self): - self.initializeWorkingDirectory() - os.chdir(self.workDirectory) - if self.switch_user: - self.switchUser() - if self.redirectIO: - sys.stdout = open('stdout', 'w') - sys.stderr = open('stderr', 'w') - command = self.commandInfo.value - env = self.setupEnvironment() - p = subprocess.Popen(['/bin/sh', '-c', command], stdout=sys.stdout, - stderr=sys.stderr, env=env) - p.wait() - - def setupEnvironment(self): - env = {} - for en in self.commandInfo.environment.variables: - env[en.name] = en.value - env['MESOS_DIRECTORY'] = self.workDirectory - env['MESOS_SLAVE_PID'] = str(self.slavepid) - env['MESOS_FRAMEWORK_ID'] = self.framework_id.value - env['MESOS_EXECUTOR_ID'] = self.executor_id.value - env['LIBPROCESS_PORT'] = '0' - return env - - def initializeWorkingDirectory(self): - if self.switch_user: - try: - os.chown(self.user, self.workDirectory) - except IOError, e: - logger.error("failed to chown: %s", e) - - def switchUser(self): - try: - pw = pwd.getpwnam(self.user) - os.setuid(pw.pw_uid) - os.setgid(pw.pw_gid) - except OSError, e: - logger.error("failed to swith to user %s: %s", self.user, e) - -def main(): - from mesos_pb2 import FrameworkID, ExecutorID, CommandInfo - fid = FrameworkID() - fid.value = os.environ.get('MESOS_FRAMEWORK_ID', 'fid') - eid = ExecutorID() - eid.value = os.environ.get('MESOS_EXECUTOR_ID', 'eid') - info = CommandInfo() - info.value = os.environ.get('MESOS_COMMAND') - return Launcher(fid, eid, info).run() - -if __name__ == '__main__': - main() diff --git a/dpark/pymesos/mesos.proto b/dpark/pymesos/mesos.proto deleted file mode 100644 index 8012873f..00000000 --- a/dpark/pymesos/mesos.proto +++ /dev/null @@ -1,652 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package mesos; - -option java_package = "org.apache.mesos"; -option java_outer_classname = "Protos"; - - -/** - * Status is used to indicate the state of the scheduler and executor - * driver after function calls. - */ -enum Status { - DRIVER_NOT_STARTED = 1; - DRIVER_RUNNING = 2; - DRIVER_ABORTED = 3; - DRIVER_STOPPED = 4; -} - - -/** - * A unique ID assigned to a framework. A framework can reuse this ID - * in order to do failover (see MesosSchedulerDriver). - */ -message FrameworkID { - required string value = 1; -} - - -/** - * A unique ID assigned to an offer. - */ -message OfferID { - required string value = 1; -} - - -/** - * A unique ID assigned to a slave. Currently, a slave gets a new ID - * whenever it (re)registers with Mesos. Framework writers shouldn't - * assume any binding between a slave ID and and a hostname. - */ -message SlaveID { - required string value = 1; -} - - -/** - * A framework generated ID to distinguish a task. The ID must remain - * unique while the task is active. However, a framework can reuse an - * ID _only_ if a previous task with the same ID has reached a - * terminal state (e.g., TASK_FINISHED, TASK_LOST, TASK_KILLED, etc.). - */ -message TaskID { - required string value = 1; -} - - -/** - * A framework generated ID to distinguish an executor. Only one - * executor with the same ID can be active on the same slave at a - * time. - */ -message ExecutorID { - required string value = 1; -} - - -/** - * A slave generated ID to distinguish a container. The ID must be unique - * between any active or completed containers on the slave. In particular, - * containers for different runs of the same (framework, executor) pair must be - * unique. - */ -message ContainerID { - required string value = 1; -} - - -/** - * Describes a framework. The user field is used to determine the - * Unix user that an executor/task should be launched as. If the user - * field is set to an empty string Mesos will automagically set it - * to the current user. Note that the ID is only available after a - * framework has registered, however, it is included here in order to - * facilitate scheduler failover (i.e., if it is set then the - * MesosSchedulerDriver expects the scheduler is performing failover). - * The amount of time that the master will wait for the scheduler to - * failover before removing the framework is specified by - * failover_timeout. If checkpoint is set, framework pid, executor - * pids and status updates are checkpointed to disk by the slaves. - * Checkpointing allows a restarted slave to reconnect with old - * executors and recover status updates, at the cost of disk I/O. - * The role field is used to group frameworks for allocation - * decisions, depending on the allocation policy being used. - * If the hostname field is set to an empty string Mesos will - * automagically set it to the current hostname. - */ -message FrameworkInfo { - required string user = 1; - required string name = 2; - optional FrameworkID id = 3; - optional double failover_timeout = 4 [default = 0.0]; - optional bool checkpoint = 5 [default = false]; - optional string role = 6 [default = "*"]; - optional string hostname = 7; -} - - -/** - * Describes a health check for a task or executor (or any arbitrary - * process/command). A "strategy" is picked by specifying one of the - * optional fields, currently only 'http' is supported. Specifying - * more than one strategy is an error. - */ -message HealthCheck { - // Describes an HTTP health check. - message HTTP { - // Port to send the HTTP request. - required uint32 port = 1; - - // HTTP request path. - optional string path = 2 [default = "/"]; - - // TODO(benh): Implement: - // Whether or not to use HTTPS. - // optional bool ssl = 3 [default = false]; - - // Expected response statuses. Not specifying any statuses implies - // that any returned status is acceptable. - repeated uint32 statuses = 4; - - // TODO(benh): Include an 'optional bytes data' field for checking - // for specific data in the response. - } - - optional HTTP http = 1; - - // TODO(benh): Consider adding a URL health check strategy which - // allows doing something similar to the HTTP strategy but - // encapsulates all the details in a single string field. - - // TODO(benh): Other possible health check strategies could include - // one for TCP/UDP or a "command". A "command" could be running a - // (shell) command to check the healthiness. We'd need to determine - // what arguments (or environment variables) we'd want to set so - // that the command could do it's job (i.e., do we want to expose - // the stdout/stderr and/or the pid to make checking for healthiness - // easier). - - // Amount of time to wait until starting the health checks. - optional double delay_seconds = 2 [default = 15.0]; - - // Interval between health checks. - optional double interval_seconds = 3 [default = 10.0]; - - // Amount of time to wait for the health check to complete. - optional double timeout_seconds = 4 [default = 20.0]; - - // Number of consecutive failures until considered unhealthy. - optional uint32 failures = 5 [default = 3]; -} - - -/** - * Describes a command, executed via: '/bin/sh -c value'. Any URIs specified - * are fetched before executing the command. If the executable field for an - * uri is set, executable file permission is set on the downloaded file. - * Otherwise, if the downloaded file has a recognized archive extension - * (currently [compressed] tar and zip) it is extracted into the executor's - * working directory. This extraction can be disabled by setting `extract` to - * false. In addition, any environment variables are set before executing - * the command (so they can be used to "parameterize" your command). - */ -message CommandInfo { - message URI { - required string value = 1; - optional bool executable = 2; - optional bool extract = 3 [default = true]; - } - - // Describes a container. - // Not all containerizers currently implement ContainerInfo, so it - // is possible that a launched task will fail due to supplying this - // attribute. - // NOTE: The containerizer API is currently in an early beta or - // even alpha state. Some details, like the exact semantics of an - // "image" or "options" are not yet hardened. - // TODO(tillt): Describe the exact scheme and semantics of "image" - // and "options". - message ContainerInfo { - // URI describing the container image name. - required string image = 1; - - // Describes additional options passed to the containerizer. - repeated string options = 2; - } - - // NOTE: MesosContainerizer does currently not support this - // attribute and tasks supplying a 'container' will fail. - optional ContainerInfo container = 4; - - repeated URI uris = 1; - - optional Environment environment = 2; - - // Actual command (i.e., 'echo hello world'). - required string value = 3; - - // Enables executor and tasks to run as a specific user. If the user - // field is present both in FrameworkInfo and here, the CommandInfo - // user value takes precedence. - optional string user = 5; - - // A health check for the command (currently in *alpha* and initial - // support will only be for TaskInfo's that have a CommandInfo). - optional HealthCheck health_check = 6; -} - - -/** - * Describes information about an executor. The 'data' field can be - * used to pass arbitrary bytes to an executor. - */ -message ExecutorInfo { - required ExecutorID executor_id = 1; - optional FrameworkID framework_id = 8; // TODO(benh): Make this required. - required CommandInfo command = 7; - repeated Resource resources = 5; - optional string name = 9; - - // Source is an identifier style string used by frameworks to track - // the source of an executor. This is useful when it's possible for - // different executor ids to be related semantically. - // NOTE: Source is exposed alongside the resource usage of the - // executor via JSON on the slave. This allows users to import - // usage information into a time series database for monitoring. - optional string source = 10; - optional bytes data = 4; -} - - -/** - * Describes a master. This will probably have more fields in the - * future which might be used, for example, to link a framework webui - * to a master webui. - */ -message MasterInfo { - required string id = 1; - required uint32 ip = 2; - required uint32 port = 3 [default = 5050]; - optional string pid = 4; - optional string hostname = 5; -} - - -/** - * Describes a slave. Note that the 'id' field is only available after - * a slave is registered with the master, and is made available here - * to facilitate re-registration. If checkpoint is set, the slave is - * checkpointing its own information and potentially frameworks' - * information (if a framework has checkpointing enabled). - */ -message SlaveInfo { - required string hostname = 1; - optional int32 port = 8 [default = 5051]; - repeated Resource resources = 3; - repeated Attribute attributes = 5; - optional SlaveID id = 6; - optional bool checkpoint = 7 [default = false]; - - // Deprecated! - optional string webui_hostname = 2; - optional int32 webui_port = 4 [default = 8081]; -} - - -/** - * Describes an Attribute or Resource "value". A value is described - * using the standard protocol buffer "union" trick. - */ -message Value { - enum Type { - SCALAR = 0; - RANGES = 1; - SET = 2; - TEXT = 3; - } - - message Scalar { - required double value = 1; - } - - message Range { - required uint64 begin = 1; - required uint64 end = 2; - } - - message Ranges { - repeated Range range = 1; - } - - message Set { - repeated string item = 1; - } - - message Text { - required string value = 1; - } - - required Type type = 1; - optional Scalar scalar = 2; - optional Ranges ranges = 3; - optional Set set = 4; - optional Text text = 5; -} - - -/** - * Describes an attribute that can be set on a machine. For now, - * attributes and resources share the same "value" type, but this may - * change in the future and attributes may only be string based. - */ -message Attribute { - required string name = 1; - required Value.Type type = 2; - optional Value.Scalar scalar = 3; - optional Value.Ranges ranges = 4; - optional Value.Set set = 6; - optional Value.Text text = 5; -} - - -/** - * Describes a resource on a machine. A resource can take on one of - * three types: scalar (double), a list of finite and discrete ranges - * (e.g., [1-10, 20-30]), or a set of items. A resource is described - * using the standard protocol buffer "union" trick. - * - * TODO(benh): Add better support for "expected" resources (e.g., - * cpus, memory, disk, network). - */ -message Resource { - required string name = 1; - required Value.Type type = 2; - optional Value.Scalar scalar = 3; - optional Value.Ranges ranges = 4; - optional Value.Set set = 5; - optional string role = 6 [default = "*"]; -} - - -/* - * A snapshot of resource usage statistics. - */ -message ResourceStatistics { - required double timestamp = 1; // Snapshot time, in seconds since the Epoch. - - // CPU Usage Information: - // Total CPU time spent in user mode, and kernel mode. - optional double cpus_user_time_secs = 2; - optional double cpus_system_time_secs = 3; - - // Number of CPUs allocated. - optional double cpus_limit = 4; - - // cpu.stat on process throttling (for contention issues). - optional uint32 cpus_nr_periods = 7; - optional uint32 cpus_nr_throttled = 8; - optional double cpus_throttled_time_secs = 9; - - // Memory Usage Information: - optional uint64 mem_rss_bytes = 5; // Resident Set Size. - - // Amount of memory resources allocated. - optional uint64 mem_limit_bytes = 6; - - // Broken out memory usage information (files, anonymous, and mmaped files) - optional uint64 mem_file_bytes = 10; - optional uint64 mem_anon_bytes = 11; - optional uint64 mem_mapped_file_bytes = 12; - - // TODO(bmahler): Add disk usage. - // TODO(bmahler): Add network usage? -} - - -/** - * Describes a snapshot of the resource usage for an executor. - * - * TODO(bmahler): Note that we want to be sending this information - * to the master, and subsequently to the relevant scheduler. So - * this proto is designed to be easy for the scheduler to use, this - * is why we provide the slave id, executor info / task info. - */ -message ResourceUsage { - required SlaveID slave_id = 1; - required FrameworkID framework_id = 2; - - // Resource usage is for an executor. For tasks launched with - // an explicit executor, the executor id is provided. For tasks - // launched without an executor, our internal executor will be - // used. In this case, we provide the task id here instead, in - // order to make this message easier for schedulers to work with. - - optional ExecutorID executor_id = 3; // If present, this executor was - optional string executor_name = 4; // explicitly specified. - - optional TaskID task_id = 5; // If present, the task did not have an executor. - - // If missing, the isolation module cannot provide resource usage. - optional ResourceStatistics statistics = 6; -} - - -/** - * Describes a request for resources that can be used by a framework - * to proactively influence the allocator. If 'slave_id' is provided - * then this request is assumed to only apply to resources on that - * slave. - */ -message Request { - optional SlaveID slave_id = 1; - repeated Resource resources = 2; -} - - -/** - * Describes some resources available on a slave. An offer only - * contains resources from a single slave. - */ -message Offer { - required OfferID id = 1; - required FrameworkID framework_id = 2; - required SlaveID slave_id = 3; - required string hostname = 4; - repeated Resource resources = 5; - repeated Attribute attributes = 7; - repeated ExecutorID executor_ids = 6; -} - - -/** - * Describes a task. Passed from the scheduler all the way to an - * executor (see SchedulerDriver::launchTasks and - * Executor::launchTask). Either ExecutorInfo or CommandInfo should be set. - * A different executor can be used to launch this task, and subsequent tasks - * meant for the same executor can reuse the same ExecutorInfo struct. - */ -message TaskInfo { - required string name = 1; - required TaskID task_id = 2; - required SlaveID slave_id = 3; - repeated Resource resources = 4; - optional ExecutorInfo executor = 5; - optional CommandInfo command = 7; - optional bytes data = 6; -} - - -/** - * Describes possible task states. IMPORTANT: Mesos assumes tasks that - * enter terminal states (see below) imply the task is no longer - * running and thus clean up any thing associated with the task - * (ultimately offering any resources being consumed by that task to - * another task). - */ -enum TaskState { - TASK_STAGING = 6; // Initial state. Framework status updates should not use. - TASK_STARTING = 0; - TASK_RUNNING = 1; - TASK_FINISHED = 2; // TERMINAL. - TASK_FAILED = 3; // TERMINAL. - TASK_KILLED = 4; // TERMINAL. - TASK_LOST = 5; // TERMINAL. -} - - -/** - * Describes the current status of a task. - */ -message TaskStatus { - required TaskID task_id = 1; - required TaskState state = 2; - optional string message = 4; // Possible message explaining state. - optional bytes data = 3; - optional SlaveID slave_id = 5; - optional double timestamp = 6; -} - - -/** - * Describes possible filters that can be applied to unused resources - * (see SchedulerDriver::launchTasks) to influence the allocator. - */ -message Filters { - // Time to consider unused resources refused. Note that all unused - // resources will be considered refused and use the default value - // (below) regardless of whether Filters was passed to - // SchedulerDriver::launchTasks. You MUST pass Filters with this - // field set to change this behavior (i.e., get another offer which - // includes unused resources sooner or later than the default). - optional double refuse_seconds = 1 [default = 5.0]; -} - - -/** -* Describes a collection of environment variables. This is used with -* CommandInfo in order to set environment variables before running a -* command. -*/ -message Environment { - message Variable { - required string name = 1; - required string value = 2; - } - - repeated Variable variables = 1; -} - - -/** - * A generic (key, value) pair used in various places for parameters. - */ -message Parameter { - required string key = 1; - required string value = 2; -} - - -/** - * Collection of Parameter. - */ -message Parameters { - repeated Parameter parameter = 1; -} - - -/** - * Credential used for authentication. - * - * NOTE: The 'principal' is used for authenticating the framework or slave - * with the master. This is different from 'FrameworkInfo.user' - * which is used to determine the user under which the framework's - * executors/tasks are run. - */ -message Credential { - required string principal = 1; - optional bytes secret = 2; -} - - -/** - * ACLs used for authorization. - */ -message ACL { - - // Entity is used to describe a subject(s) or an object(s) of an ACL. - // NOTE: - // To allow everyone access to an Entity set its type to 'ANY'. - // To deny access to an Entity set its type to 'NONE'. - message Entity { - enum Type { - SOME = 0; - ANY = 1; - NONE = 2; - } - optional Type type = 1 [default = SOME]; - repeated string values = 2; // Ignored for ANY/NONE. - } - - // ACLs. - message RunTasks { - // Subjects. - required Entity principals = 1; // Framework principals. - - // Objects. - required Entity users = 2; // Users to run the tasks/executors as. - } - - message ReceiveOffers { - // Subjects. - required Entity principals = 1; // Framework principals. - - // Objects. - required Entity roles = 2; // Resource roles that can be offered. - } - - message HTTPGet { - // Subjects (At least one of these should be set). - optional Entity usernames = 1; // HTTP authentication based usernames. - optional Entity ips = 2; - optional Entity hostnames = 3; - - // Objects. - required Entity urls = 4; - } - - message HTTPPut { - // Subjects (At least one of these should be set). - optional Entity usernames = 1; // HTTP authentication based usernames. - optional Entity ips = 2; - optional Entity hostnames = 3; - - // Objects. - required Entity urls = 4; - } -} - - -/* - * Collection of ACL. - * - * Each authorization request is evaluated against the ACLs in the order - * they are defined. - * - * For simplicity, the ACLs for a given action are not aggregated even - * when they have the same subjects or objects. The first ACL that - * matches the request determines whether that request should be - * permitted or not. An ACL matches iff both the subjects - * (e.g., clients, principals) and the objects (e.g., urls, users, - * roles) of the ACL match the request. - * - * If none of the ACLs match the request, the 'permissive' field - * determines whether the request should be permitted or not. - * - * TODO(vinod): Do aggregation of ACLs when possible. - * - */ -message ACLs { - required bool permissive = 1 [default = true]; - repeated ACL.RunTasks run_tasks = 2; - repeated ACL.ReceiveOffers receive_offers = 3; - repeated ACL.HTTPGet http_get = 4; - repeated ACL.HTTPPut http_put = 5; -} diff --git a/dpark/pymesos/mesos_pb2.py b/dpark/pymesos/mesos_pb2.py deleted file mode 100644 index 334f17ba..00000000 --- a/dpark/pymesos/mesos_pb2.py +++ /dev/null @@ -1,2391 +0,0 @@ -# Generated by the protocol buffer compiler. DO NOT EDIT! - -from google.protobuf import descriptor -from google.protobuf import message -from google.protobuf import reflection -from google.protobuf import descriptor_pb2 -# @@protoc_insertion_point(imports) - - - -DESCRIPTOR = descriptor.FileDescriptor( - name='mesos.proto', - package='mesos', - serialized_pb='\n\x0bmesos.proto\x12\x05mesos\"\x1c\n\x0b\x46rameworkID\x12\r\n\x05value\x18\x01 \x02(\t\"\x18\n\x07OfferID\x12\r\n\x05value\x18\x01 \x02(\t\"\x18\n\x07SlaveID\x12\r\n\x05value\x18\x01 \x02(\t\"\x17\n\x06TaskID\x12\r\n\x05value\x18\x01 \x02(\t\"\x1b\n\nExecutorID\x12\r\n\x05value\x18\x01 \x02(\t\"\x1c\n\x0b\x43ontainerID\x12\r\n\x05value\x18\x01 \x02(\t\"\xa6\x01\n\rFrameworkInfo\x12\x0c\n\x04user\x18\x01 \x02(\t\x12\x0c\n\x04name\x18\x02 \x02(\t\x12\x1e\n\x02id\x18\x03 \x01(\x0b\x32\x12.mesos.FrameworkID\x12\x1b\n\x10\x66\x61ilover_timeout\x18\x04 \x01(\x01:\x01\x30\x12\x19\n\ncheckpoint\x18\x05 \x01(\x08:\x05\x66\x61lse\x12\x0f\n\x04role\x18\x06 \x01(\t:\x01*\x12\x10\n\x08hostname\x18\x07 \x01(\t\"\xd8\x01\n\x0bHealthCheck\x12%\n\x04http\x18\x01 \x01(\x0b\x32\x17.mesos.HealthCheck.HTTP\x12\x19\n\rdelay_seconds\x18\x02 \x01(\x01:\x02\x31\x35\x12\x1c\n\x10interval_seconds\x18\x03 \x01(\x01:\x02\x31\x30\x12\x1b\n\x0ftimeout_seconds\x18\x04 \x01(\x01:\x02\x32\x30\x12\x13\n\x08\x66\x61ilures\x18\x05 \x01(\r:\x01\x33\x1a\x37\n\x04HTTP\x12\x0c\n\x04port\x18\x01 \x02(\r\x12\x0f\n\x04path\x18\x02 \x01(\t:\x01/\x12\x10\n\x08statuses\x18\x04 \x03(\r\"\xca\x02\n\x0b\x43ommandInfo\x12\x33\n\tcontainer\x18\x04 \x01(\x0b\x32 .mesos.CommandInfo.ContainerInfo\x12$\n\x04uris\x18\x01 \x03(\x0b\x32\x16.mesos.CommandInfo.URI\x12\'\n\x0b\x65nvironment\x18\x02 \x01(\x0b\x32\x12.mesos.Environment\x12\r\n\x05value\x18\x03 \x02(\t\x12\x0c\n\x04user\x18\x05 \x01(\t\x12(\n\x0chealth_check\x18\x06 \x01(\x0b\x32\x12.mesos.HealthCheck\x1a?\n\x03URI\x12\r\n\x05value\x18\x01 \x02(\t\x12\x12\n\nexecutable\x18\x02 \x01(\x08\x12\x15\n\x07\x65xtract\x18\x03 \x01(\x08:\x04true\x1a/\n\rContainerInfo\x12\r\n\x05image\x18\x01 \x02(\t\x12\x0f\n\x07options\x18\x02 \x03(\t\"\xd5\x01\n\x0c\x45xecutorInfo\x12&\n\x0b\x65xecutor_id\x18\x01 \x02(\x0b\x32\x11.mesos.ExecutorID\x12(\n\x0c\x66ramework_id\x18\x08 \x01(\x0b\x32\x12.mesos.FrameworkID\x12#\n\x07\x63ommand\x18\x07 \x02(\x0b\x32\x12.mesos.CommandInfo\x12\"\n\tresources\x18\x05 \x03(\x0b\x32\x0f.mesos.Resource\x12\x0c\n\x04name\x18\t \x01(\t\x12\x0e\n\x06source\x18\n \x01(\t\x12\x0c\n\x04\x64\x61ta\x18\x04 \x01(\x0c\"W\n\nMasterInfo\x12\n\n\x02id\x18\x01 \x02(\t\x12\n\n\x02ip\x18\x02 \x02(\r\x12\x12\n\x04port\x18\x03 \x02(\r:\x04\x35\x30\x35\x30\x12\x0b\n\x03pid\x18\x04 \x01(\t\x12\x10\n\x08hostname\x18\x05 \x01(\t\"\xe4\x01\n\tSlaveInfo\x12\x10\n\x08hostname\x18\x01 \x02(\t\x12\x12\n\x04port\x18\x08 \x01(\x05:\x04\x35\x30\x35\x31\x12\"\n\tresources\x18\x03 \x03(\x0b\x32\x0f.mesos.Resource\x12$\n\nattributes\x18\x05 \x03(\x0b\x32\x10.mesos.Attribute\x12\x1a\n\x02id\x18\x06 \x01(\x0b\x32\x0e.mesos.SlaveID\x12\x19\n\ncheckpoint\x18\x07 \x01(\x08:\x05\x66\x61lse\x12\x16\n\x0ewebui_hostname\x18\x02 \x01(\t\x12\x18\n\nwebui_port\x18\x04 \x01(\x05:\x04\x38\x30\x38\x31\"\xfc\x02\n\x05Value\x12\x1f\n\x04type\x18\x01 \x02(\x0e\x32\x11.mesos.Value.Type\x12#\n\x06scalar\x18\x02 \x01(\x0b\x32\x13.mesos.Value.Scalar\x12#\n\x06ranges\x18\x03 \x01(\x0b\x32\x13.mesos.Value.Ranges\x12\x1d\n\x03set\x18\x04 \x01(\x0b\x32\x10.mesos.Value.Set\x12\x1f\n\x04text\x18\x05 \x01(\x0b\x32\x11.mesos.Value.Text\x1a\x17\n\x06Scalar\x12\r\n\x05value\x18\x01 \x02(\x01\x1a#\n\x05Range\x12\r\n\x05\x62\x65gin\x18\x01 \x02(\x04\x12\x0b\n\x03\x65nd\x18\x02 \x02(\x04\x1a+\n\x06Ranges\x12!\n\x05range\x18\x01 \x03(\x0b\x32\x12.mesos.Value.Range\x1a\x13\n\x03Set\x12\x0c\n\x04item\x18\x01 \x03(\t\x1a\x15\n\x04Text\x12\r\n\x05value\x18\x01 \x02(\t\"1\n\x04Type\x12\n\n\x06SCALAR\x10\x00\x12\n\n\x06RANGES\x10\x01\x12\x07\n\x03SET\x10\x02\x12\x08\n\x04TEXT\x10\x03\"\xc4\x01\n\tAttribute\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x1f\n\x04type\x18\x02 \x02(\x0e\x32\x11.mesos.Value.Type\x12#\n\x06scalar\x18\x03 \x01(\x0b\x32\x13.mesos.Value.Scalar\x12#\n\x06ranges\x18\x04 \x01(\x0b\x32\x13.mesos.Value.Ranges\x12\x1d\n\x03set\x18\x06 \x01(\x0b\x32\x10.mesos.Value.Set\x12\x1f\n\x04text\x18\x05 \x01(\x0b\x32\x11.mesos.Value.Text\"\xb3\x01\n\x08Resource\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x1f\n\x04type\x18\x02 \x02(\x0e\x32\x11.mesos.Value.Type\x12#\n\x06scalar\x18\x03 \x01(\x0b\x32\x13.mesos.Value.Scalar\x12#\n\x06ranges\x18\x04 \x01(\x0b\x32\x13.mesos.Value.Ranges\x12\x1d\n\x03set\x18\x05 \x01(\x0b\x32\x10.mesos.Value.Set\x12\x0f\n\x04role\x18\x06 \x01(\t:\x01*\"\xcc\x02\n\x12ResourceStatistics\x12\x11\n\ttimestamp\x18\x01 \x02(\x01\x12\x1b\n\x13\x63pus_user_time_secs\x18\x02 \x01(\x01\x12\x1d\n\x15\x63pus_system_time_secs\x18\x03 \x01(\x01\x12\x12\n\ncpus_limit\x18\x04 \x01(\x01\x12\x17\n\x0f\x63pus_nr_periods\x18\x07 \x01(\r\x12\x19\n\x11\x63pus_nr_throttled\x18\x08 \x01(\r\x12 \n\x18\x63pus_throttled_time_secs\x18\t \x01(\x01\x12\x15\n\rmem_rss_bytes\x18\x05 \x01(\x04\x12\x17\n\x0fmem_limit_bytes\x18\x06 \x01(\x04\x12\x16\n\x0emem_file_bytes\x18\n \x01(\x04\x12\x16\n\x0emem_anon_bytes\x18\x0b \x01(\x04\x12\x1d\n\x15mem_mapped_file_bytes\x18\x0c \x01(\x04\"\xe9\x01\n\rResourceUsage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x03 \x01(\x0b\x32\x11.mesos.ExecutorID\x12\x15\n\rexecutor_name\x18\x04 \x01(\t\x12\x1e\n\x07task_id\x18\x05 \x01(\x0b\x32\r.mesos.TaskID\x12-\n\nstatistics\x18\x06 \x01(\x0b\x32\x19.mesos.ResourceStatistics\"O\n\x07Request\x12 \n\x08slave_id\x18\x01 \x01(\x0b\x32\x0e.mesos.SlaveID\x12\"\n\tresources\x18\x02 \x03(\x0b\x32\x0f.mesos.Resource\"\xf4\x01\n\x05Offer\x12\x1a\n\x02id\x18\x01 \x02(\x0b\x32\x0e.mesos.OfferID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12 \n\x08slave_id\x18\x03 \x02(\x0b\x32\x0e.mesos.SlaveID\x12\x10\n\x08hostname\x18\x04 \x02(\t\x12\"\n\tresources\x18\x05 \x03(\x0b\x32\x0f.mesos.Resource\x12$\n\nattributes\x18\x07 \x03(\x0b\x32\x10.mesos.Attribute\x12\'\n\x0c\x65xecutor_ids\x18\x06 \x03(\x0b\x32\x11.mesos.ExecutorID\"\xd8\x01\n\x08TaskInfo\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x1e\n\x07task_id\x18\x02 \x02(\x0b\x32\r.mesos.TaskID\x12 \n\x08slave_id\x18\x03 \x02(\x0b\x32\x0e.mesos.SlaveID\x12\"\n\tresources\x18\x04 \x03(\x0b\x32\x0f.mesos.Resource\x12%\n\x08\x65xecutor\x18\x05 \x01(\x0b\x32\x13.mesos.ExecutorInfo\x12#\n\x07\x63ommand\x18\x07 \x01(\x0b\x32\x12.mesos.CommandInfo\x12\x0c\n\x04\x64\x61ta\x18\x06 \x01(\x0c\"\xa1\x01\n\nTaskStatus\x12\x1e\n\x07task_id\x18\x01 \x02(\x0b\x32\r.mesos.TaskID\x12\x1f\n\x05state\x18\x02 \x02(\x0e\x32\x10.mesos.TaskState\x12\x0f\n\x07message\x18\x04 \x01(\t\x12\x0c\n\x04\x64\x61ta\x18\x03 \x01(\x0c\x12 \n\x08slave_id\x18\x05 \x01(\x0b\x32\x0e.mesos.SlaveID\x12\x11\n\ttimestamp\x18\x06 \x01(\x01\"$\n\x07\x46ilters\x12\x19\n\x0erefuse_seconds\x18\x01 \x01(\x01:\x01\x35\"f\n\x0b\x45nvironment\x12.\n\tvariables\x18\x01 \x03(\x0b\x32\x1b.mesos.Environment.Variable\x1a\'\n\x08Variable\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\r\n\x05value\x18\x02 \x02(\t\"\'\n\tParameter\x12\x0b\n\x03key\x18\x01 \x02(\t\x12\r\n\x05value\x18\x02 \x02(\t\"1\n\nParameters\x12#\n\tparameter\x18\x01 \x03(\x0b\x32\x10.mesos.Parameter\"/\n\nCredential\x12\x11\n\tprincipal\x18\x01 \x02(\t\x12\x0e\n\x06secret\x18\x02 \x01(\x0c\"\xd1\x04\n\x03\x41\x43L\x1ai\n\x06\x45ntity\x12*\n\x04type\x18\x01 \x01(\x0e\x32\x16.mesos.ACL.Entity.Type:\x04SOME\x12\x0e\n\x06values\x18\x02 \x03(\t\"#\n\x04Type\x12\x08\n\x04SOME\x10\x00\x12\x07\n\x03\x41NY\x10\x01\x12\x08\n\x04NONE\x10\x02\x1aS\n\x08RunTasks\x12%\n\nprincipals\x18\x01 \x02(\x0b\x32\x11.mesos.ACL.Entity\x12 \n\x05users\x18\x02 \x02(\x0b\x32\x11.mesos.ACL.Entity\x1aX\n\rReceiveOffers\x12%\n\nprincipals\x18\x01 \x02(\x0b\x32\x11.mesos.ACL.Entity\x12 \n\x05roles\x18\x02 \x02(\x0b\x32\x11.mesos.ACL.Entity\x1a\x96\x01\n\x07HTTPGet\x12$\n\tusernames\x18\x01 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12\x1e\n\x03ips\x18\x02 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12$\n\thostnames\x18\x03 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12\x1f\n\x04urls\x18\x04 \x02(\x0b\x32\x11.mesos.ACL.Entity\x1a\x96\x01\n\x07HTTPPut\x12$\n\tusernames\x18\x01 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12\x1e\n\x03ips\x18\x02 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12$\n\thostnames\x18\x03 \x01(\x0b\x32\x11.mesos.ACL.Entity\x12\x1f\n\x04urls\x18\x04 \x02(\x0b\x32\x11.mesos.ACL.Entity\"\xc6\x01\n\x04\x41\x43Ls\x12\x18\n\npermissive\x18\x01 \x02(\x08:\x04true\x12&\n\trun_tasks\x18\x02 \x03(\x0b\x32\x13.mesos.ACL.RunTasks\x12\x30\n\x0ereceive_offers\x18\x03 \x03(\x0b\x32\x18.mesos.ACL.ReceiveOffers\x12$\n\x08http_get\x18\x04 \x03(\x0b\x32\x12.mesos.ACL.HTTPGet\x12$\n\x08http_put\x18\x05 \x03(\x0b\x32\x12.mesos.ACL.HTTPPut*\\\n\x06Status\x12\x16\n\x12\x44RIVER_NOT_STARTED\x10\x01\x12\x12\n\x0e\x44RIVER_RUNNING\x10\x02\x12\x12\n\x0e\x44RIVER_ABORTED\x10\x03\x12\x12\n\x0e\x44RIVER_STOPPED\x10\x04*\x86\x01\n\tTaskState\x12\x10\n\x0cTASK_STAGING\x10\x06\x12\x11\n\rTASK_STARTING\x10\x00\x12\x10\n\x0cTASK_RUNNING\x10\x01\x12\x11\n\rTASK_FINISHED\x10\x02\x12\x0f\n\x0bTASK_FAILED\x10\x03\x12\x0f\n\x0bTASK_KILLED\x10\x04\x12\r\n\tTASK_LOST\x10\x05\x42\x1a\n\x10org.apache.mesosB\x06Protos') - -_STATUS = descriptor.EnumDescriptor( - name='Status', - full_name='mesos.Status', - filename=None, - file=DESCRIPTOR, - values=[ - descriptor.EnumValueDescriptor( - name='DRIVER_NOT_STARTED', index=0, number=1, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='DRIVER_RUNNING', index=1, number=2, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='DRIVER_ABORTED', index=2, number=3, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='DRIVER_STOPPED', index=3, number=4, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=4571, - serialized_end=4663, -) - - -_TASKSTATE = descriptor.EnumDescriptor( - name='TaskState', - full_name='mesos.TaskState', - filename=None, - file=DESCRIPTOR, - values=[ - descriptor.EnumValueDescriptor( - name='TASK_STAGING', index=0, number=6, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_STARTING', index=1, number=0, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_RUNNING', index=2, number=1, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_FINISHED', index=3, number=2, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_FAILED', index=4, number=3, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_KILLED', index=5, number=4, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TASK_LOST', index=6, number=5, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=4666, - serialized_end=4800, -) - - -DRIVER_NOT_STARTED = 1 -DRIVER_RUNNING = 2 -DRIVER_ABORTED = 3 -DRIVER_STOPPED = 4 -TASK_STAGING = 6 -TASK_STARTING = 0 -TASK_RUNNING = 1 -TASK_FINISHED = 2 -TASK_FAILED = 3 -TASK_KILLED = 4 -TASK_LOST = 5 - - -_VALUE_TYPE = descriptor.EnumDescriptor( - name='Type', - full_name='mesos.Value.Type', - filename=None, - file=DESCRIPTOR, - values=[ - descriptor.EnumValueDescriptor( - name='SCALAR', index=0, number=0, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='RANGES', index=1, number=1, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='SET', index=2, number=2, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='TEXT', index=3, number=3, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=1777, - serialized_end=1826, -) - -_ACL_ENTITY_TYPE = descriptor.EnumDescriptor( - name='Type', - full_name='mesos.ACL.Entity.Type', - filename=None, - file=DESCRIPTOR, - values=[ - descriptor.EnumValueDescriptor( - name='SOME', index=0, number=0, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='ANY', index=1, number=1, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='NONE', index=2, number=2, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=3852, - serialized_end=3887, -) - - -_FRAMEWORKID = descriptor.Descriptor( - name='FrameworkID', - full_name='mesos.FrameworkID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.FrameworkID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=22, - serialized_end=50, -) - - -_OFFERID = descriptor.Descriptor( - name='OfferID', - full_name='mesos.OfferID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.OfferID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=52, - serialized_end=76, -) - - -_SLAVEID = descriptor.Descriptor( - name='SlaveID', - full_name='mesos.SlaveID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.SlaveID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=78, - serialized_end=102, -) - - -_TASKID = descriptor.Descriptor( - name='TaskID', - full_name='mesos.TaskID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.TaskID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=104, - serialized_end=127, -) - - -_EXECUTORID = descriptor.Descriptor( - name='ExecutorID', - full_name='mesos.ExecutorID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.ExecutorID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=129, - serialized_end=156, -) - - -_CONTAINERID = descriptor.Descriptor( - name='ContainerID', - full_name='mesos.ContainerID', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.ContainerID.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=158, - serialized_end=186, -) - - -_FRAMEWORKINFO = descriptor.Descriptor( - name='FrameworkInfo', - full_name='mesos.FrameworkInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='user', full_name='mesos.FrameworkInfo.user', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='name', full_name='mesos.FrameworkInfo.name', index=1, - number=2, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='id', full_name='mesos.FrameworkInfo.id', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='failover_timeout', full_name='mesos.FrameworkInfo.failover_timeout', index=3, - number=4, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='checkpoint', full_name='mesos.FrameworkInfo.checkpoint', index=4, - number=5, type=8, cpp_type=7, label=1, - has_default_value=True, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='role', full_name='mesos.FrameworkInfo.role', index=5, - number=6, type=9, cpp_type=9, label=1, - has_default_value=True, default_value=unicode("*", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='hostname', full_name='mesos.FrameworkInfo.hostname', index=6, - number=7, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=189, - serialized_end=355, -) - - -_HEALTHCHECK_HTTP = descriptor.Descriptor( - name='HTTP', - full_name='mesos.HealthCheck.HTTP', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='port', full_name='mesos.HealthCheck.HTTP.port', index=0, - number=1, type=13, cpp_type=3, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='path', full_name='mesos.HealthCheck.HTTP.path', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=True, default_value=unicode("/", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='statuses', full_name='mesos.HealthCheck.HTTP.statuses', index=2, - number=4, type=13, cpp_type=3, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=519, - serialized_end=574, -) - -_HEALTHCHECK = descriptor.Descriptor( - name='HealthCheck', - full_name='mesos.HealthCheck', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='http', full_name='mesos.HealthCheck.http', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='delay_seconds', full_name='mesos.HealthCheck.delay_seconds', index=1, - number=2, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=15, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='interval_seconds', full_name='mesos.HealthCheck.interval_seconds', index=2, - number=3, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=10, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='timeout_seconds', full_name='mesos.HealthCheck.timeout_seconds', index=3, - number=4, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=20, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='failures', full_name='mesos.HealthCheck.failures', index=4, - number=5, type=13, cpp_type=3, label=1, - has_default_value=True, default_value=3, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_HEALTHCHECK_HTTP, ], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=358, - serialized_end=574, -) - - -_COMMANDINFO_URI = descriptor.Descriptor( - name='URI', - full_name='mesos.CommandInfo.URI', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.CommandInfo.URI.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executable', full_name='mesos.CommandInfo.URI.executable', index=1, - number=2, type=8, cpp_type=7, label=1, - has_default_value=False, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='extract', full_name='mesos.CommandInfo.URI.extract', index=2, - number=3, type=8, cpp_type=7, label=1, - has_default_value=True, default_value=True, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=795, - serialized_end=858, -) - -_COMMANDINFO_CONTAINERINFO = descriptor.Descriptor( - name='ContainerInfo', - full_name='mesos.CommandInfo.ContainerInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='image', full_name='mesos.CommandInfo.ContainerInfo.image', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='options', full_name='mesos.CommandInfo.ContainerInfo.options', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=860, - serialized_end=907, -) - -_COMMANDINFO = descriptor.Descriptor( - name='CommandInfo', - full_name='mesos.CommandInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='container', full_name='mesos.CommandInfo.container', index=0, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='uris', full_name='mesos.CommandInfo.uris', index=1, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='environment', full_name='mesos.CommandInfo.environment', index=2, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='value', full_name='mesos.CommandInfo.value', index=3, - number=3, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='user', full_name='mesos.CommandInfo.user', index=4, - number=5, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='health_check', full_name='mesos.CommandInfo.health_check', index=5, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_COMMANDINFO_URI, _COMMANDINFO_CONTAINERINFO, ], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=577, - serialized_end=907, -) - - -_EXECUTORINFO = descriptor.Descriptor( - name='ExecutorInfo', - full_name='mesos.ExecutorInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.ExecutorInfo.executor_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.ExecutorInfo.framework_id', index=1, - number=8, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='command', full_name='mesos.ExecutorInfo.command', index=2, - number=7, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.ExecutorInfo.resources', index=3, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='name', full_name='mesos.ExecutorInfo.name', index=4, - number=9, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='source', full_name='mesos.ExecutorInfo.source', index=5, - number=10, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.ExecutorInfo.data', index=6, - number=4, type=12, cpp_type=9, label=1, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=910, - serialized_end=1123, -) - - -_MASTERINFO = descriptor.Descriptor( - name='MasterInfo', - full_name='mesos.MasterInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='id', full_name='mesos.MasterInfo.id', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ip', full_name='mesos.MasterInfo.ip', index=1, - number=2, type=13, cpp_type=3, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='port', full_name='mesos.MasterInfo.port', index=2, - number=3, type=13, cpp_type=3, label=2, - has_default_value=True, default_value=5050, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pid', full_name='mesos.MasterInfo.pid', index=3, - number=4, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='hostname', full_name='mesos.MasterInfo.hostname', index=4, - number=5, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1125, - serialized_end=1212, -) - - -_SLAVEINFO = descriptor.Descriptor( - name='SlaveInfo', - full_name='mesos.SlaveInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='hostname', full_name='mesos.SlaveInfo.hostname', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='port', full_name='mesos.SlaveInfo.port', index=1, - number=8, type=5, cpp_type=1, label=1, - has_default_value=True, default_value=5051, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.SlaveInfo.resources', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='attributes', full_name='mesos.SlaveInfo.attributes', index=3, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='id', full_name='mesos.SlaveInfo.id', index=4, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='checkpoint', full_name='mesos.SlaveInfo.checkpoint', index=5, - number=7, type=8, cpp_type=7, label=1, - has_default_value=True, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='webui_hostname', full_name='mesos.SlaveInfo.webui_hostname', index=6, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='webui_port', full_name='mesos.SlaveInfo.webui_port', index=7, - number=4, type=5, cpp_type=1, label=1, - has_default_value=True, default_value=8081, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1215, - serialized_end=1443, -) - - -_VALUE_SCALAR = descriptor.Descriptor( - name='Scalar', - full_name='mesos.Value.Scalar', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.Value.Scalar.value', index=0, - number=1, type=1, cpp_type=5, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1626, - serialized_end=1649, -) - -_VALUE_RANGE = descriptor.Descriptor( - name='Range', - full_name='mesos.Value.Range', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='begin', full_name='mesos.Value.Range.begin', index=0, - number=1, type=4, cpp_type=4, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='end', full_name='mesos.Value.Range.end', index=1, - number=2, type=4, cpp_type=4, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1651, - serialized_end=1686, -) - -_VALUE_RANGES = descriptor.Descriptor( - name='Ranges', - full_name='mesos.Value.Ranges', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='range', full_name='mesos.Value.Ranges.range', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1688, - serialized_end=1731, -) - -_VALUE_SET = descriptor.Descriptor( - name='Set', - full_name='mesos.Value.Set', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='item', full_name='mesos.Value.Set.item', index=0, - number=1, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1733, - serialized_end=1752, -) - -_VALUE_TEXT = descriptor.Descriptor( - name='Text', - full_name='mesos.Value.Text', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='value', full_name='mesos.Value.Text.value', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1754, - serialized_end=1775, -) - -_VALUE = descriptor.Descriptor( - name='Value', - full_name='mesos.Value', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='type', full_name='mesos.Value.type', index=0, - number=1, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='scalar', full_name='mesos.Value.scalar', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ranges', full_name='mesos.Value.ranges', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='set', full_name='mesos.Value.set', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='text', full_name='mesos.Value.text', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_VALUE_SCALAR, _VALUE_RANGE, _VALUE_RANGES, _VALUE_SET, _VALUE_TEXT, ], - enum_types=[ - _VALUE_TYPE, - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1446, - serialized_end=1826, -) - - -_ATTRIBUTE = descriptor.Descriptor( - name='Attribute', - full_name='mesos.Attribute', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.Attribute.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='type', full_name='mesos.Attribute.type', index=1, - number=2, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='scalar', full_name='mesos.Attribute.scalar', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ranges', full_name='mesos.Attribute.ranges', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='set', full_name='mesos.Attribute.set', index=4, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='text', full_name='mesos.Attribute.text', index=5, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1829, - serialized_end=2025, -) - - -_RESOURCE = descriptor.Descriptor( - name='Resource', - full_name='mesos.Resource', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.Resource.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='type', full_name='mesos.Resource.type', index=1, - number=2, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='scalar', full_name='mesos.Resource.scalar', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ranges', full_name='mesos.Resource.ranges', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='set', full_name='mesos.Resource.set', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='role', full_name='mesos.Resource.role', index=5, - number=6, type=9, cpp_type=9, label=1, - has_default_value=True, default_value=unicode("*", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2028, - serialized_end=2207, -) - - -_RESOURCESTATISTICS = descriptor.Descriptor( - name='ResourceStatistics', - full_name='mesos.ResourceStatistics', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='timestamp', full_name='mesos.ResourceStatistics.timestamp', index=0, - number=1, type=1, cpp_type=5, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_user_time_secs', full_name='mesos.ResourceStatistics.cpus_user_time_secs', index=1, - number=2, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_system_time_secs', full_name='mesos.ResourceStatistics.cpus_system_time_secs', index=2, - number=3, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_limit', full_name='mesos.ResourceStatistics.cpus_limit', index=3, - number=4, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_nr_periods', full_name='mesos.ResourceStatistics.cpus_nr_periods', index=4, - number=7, type=13, cpp_type=3, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_nr_throttled', full_name='mesos.ResourceStatistics.cpus_nr_throttled', index=5, - number=8, type=13, cpp_type=3, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='cpus_throttled_time_secs', full_name='mesos.ResourceStatistics.cpus_throttled_time_secs', index=6, - number=9, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='mem_rss_bytes', full_name='mesos.ResourceStatistics.mem_rss_bytes', index=7, - number=5, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='mem_limit_bytes', full_name='mesos.ResourceStatistics.mem_limit_bytes', index=8, - number=6, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='mem_file_bytes', full_name='mesos.ResourceStatistics.mem_file_bytes', index=9, - number=10, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='mem_anon_bytes', full_name='mesos.ResourceStatistics.mem_anon_bytes', index=10, - number=11, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='mem_mapped_file_bytes', full_name='mesos.ResourceStatistics.mem_mapped_file_bytes', index=11, - number=12, type=4, cpp_type=4, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2210, - serialized_end=2542, -) - - -_RESOURCEUSAGE = descriptor.Descriptor( - name='ResourceUsage', - full_name='mesos.ResourceUsage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.ResourceUsage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.ResourceUsage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.ResourceUsage.executor_id', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_name', full_name='mesos.ResourceUsage.executor_name', index=3, - number=4, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.ResourceUsage.task_id', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='statistics', full_name='mesos.ResourceUsage.statistics', index=5, - number=6, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2545, - serialized_end=2778, -) - - -_REQUEST = descriptor.Descriptor( - name='Request', - full_name='mesos.Request', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.Request.slave_id', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.Request.resources', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2780, - serialized_end=2859, -) - - -_OFFER = descriptor.Descriptor( - name='Offer', - full_name='mesos.Offer', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='id', full_name='mesos.Offer.id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.Offer.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.Offer.slave_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='hostname', full_name='mesos.Offer.hostname', index=3, - number=4, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.Offer.resources', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='attributes', full_name='mesos.Offer.attributes', index=5, - number=7, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_ids', full_name='mesos.Offer.executor_ids', index=6, - number=6, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2862, - serialized_end=3106, -) - - -_TASKINFO = descriptor.Descriptor( - name='TaskInfo', - full_name='mesos.TaskInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.TaskInfo.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.TaskInfo.task_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.TaskInfo.slave_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.TaskInfo.resources', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor', full_name='mesos.TaskInfo.executor', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='command', full_name='mesos.TaskInfo.command', index=5, - number=7, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.TaskInfo.data', index=6, - number=6, type=12, cpp_type=9, label=1, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3109, - serialized_end=3325, -) - - -_TASKSTATUS = descriptor.Descriptor( - name='TaskStatus', - full_name='mesos.TaskStatus', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.TaskStatus.task_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='state', full_name='mesos.TaskStatus.state', index=1, - number=2, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=6, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='message', full_name='mesos.TaskStatus.message', index=2, - number=4, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.TaskStatus.data', index=3, - number=3, type=12, cpp_type=9, label=1, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.TaskStatus.slave_id', index=4, - number=5, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='timestamp', full_name='mesos.TaskStatus.timestamp', index=5, - number=6, type=1, cpp_type=5, label=1, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3328, - serialized_end=3489, -) - - -_FILTERS = descriptor.Descriptor( - name='Filters', - full_name='mesos.Filters', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='refuse_seconds', full_name='mesos.Filters.refuse_seconds', index=0, - number=1, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=5, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3491, - serialized_end=3527, -) - - -_ENVIRONMENT_VARIABLE = descriptor.Descriptor( - name='Variable', - full_name='mesos.Environment.Variable', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.Environment.Variable.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='value', full_name='mesos.Environment.Variable.value', index=1, - number=2, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3592, - serialized_end=3631, -) - -_ENVIRONMENT = descriptor.Descriptor( - name='Environment', - full_name='mesos.Environment', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='variables', full_name='mesos.Environment.variables', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_ENVIRONMENT_VARIABLE, ], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3529, - serialized_end=3631, -) - - -_PARAMETER = descriptor.Descriptor( - name='Parameter', - full_name='mesos.Parameter', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='key', full_name='mesos.Parameter.key', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='value', full_name='mesos.Parameter.value', index=1, - number=2, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3633, - serialized_end=3672, -) - - -_PARAMETERS = descriptor.Descriptor( - name='Parameters', - full_name='mesos.Parameters', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='parameter', full_name='mesos.Parameters.parameter', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3674, - serialized_end=3723, -) - - -_CREDENTIAL = descriptor.Descriptor( - name='Credential', - full_name='mesos.Credential', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='principal', full_name='mesos.Credential.principal', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='secret', full_name='mesos.Credential.secret', index=1, - number=2, type=12, cpp_type=9, label=1, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3725, - serialized_end=3772, -) - - -_ACL_ENTITY = descriptor.Descriptor( - name='Entity', - full_name='mesos.ACL.Entity', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='type', full_name='mesos.ACL.Entity.type', index=0, - number=1, type=14, cpp_type=8, label=1, - has_default_value=True, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='values', full_name='mesos.ACL.Entity.values', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - _ACL_ENTITY_TYPE, - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3782, - serialized_end=3887, -) - -_ACL_RUNTASKS = descriptor.Descriptor( - name='RunTasks', - full_name='mesos.ACL.RunTasks', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='principals', full_name='mesos.ACL.RunTasks.principals', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='users', full_name='mesos.ACL.RunTasks.users', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3889, - serialized_end=3972, -) - -_ACL_RECEIVEOFFERS = descriptor.Descriptor( - name='ReceiveOffers', - full_name='mesos.ACL.ReceiveOffers', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='principals', full_name='mesos.ACL.ReceiveOffers.principals', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='roles', full_name='mesos.ACL.ReceiveOffers.roles', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3974, - serialized_end=4062, -) - -_ACL_HTTPGET = descriptor.Descriptor( - name='HTTPGet', - full_name='mesos.ACL.HTTPGet', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='usernames', full_name='mesos.ACL.HTTPGet.usernames', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ips', full_name='mesos.ACL.HTTPGet.ips', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='hostnames', full_name='mesos.ACL.HTTPGet.hostnames', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='urls', full_name='mesos.ACL.HTTPGet.urls', index=3, - number=4, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4065, - serialized_end=4215, -) - -_ACL_HTTPPUT = descriptor.Descriptor( - name='HTTPPut', - full_name='mesos.ACL.HTTPPut', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='usernames', full_name='mesos.ACL.HTTPPut.usernames', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='ips', full_name='mesos.ACL.HTTPPut.ips', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='hostnames', full_name='mesos.ACL.HTTPPut.hostnames', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='urls', full_name='mesos.ACL.HTTPPut.urls', index=3, - number=4, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4218, - serialized_end=4368, -) - -_ACL = descriptor.Descriptor( - name='ACL', - full_name='mesos.ACL', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[_ACL_ENTITY, _ACL_RUNTASKS, _ACL_RECEIVEOFFERS, _ACL_HTTPGET, _ACL_HTTPPUT, ], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3775, - serialized_end=4368, -) - - -_ACLS = descriptor.Descriptor( - name='ACLs', - full_name='mesos.ACLs', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='permissive', full_name='mesos.ACLs.permissive', index=0, - number=1, type=8, cpp_type=7, label=2, - has_default_value=True, default_value=True, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='run_tasks', full_name='mesos.ACLs.run_tasks', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='receive_offers', full_name='mesos.ACLs.receive_offers', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='http_get', full_name='mesos.ACLs.http_get', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='http_put', full_name='mesos.ACLs.http_put', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4371, - serialized_end=4569, -) - -_FRAMEWORKINFO.fields_by_name['id'].message_type = _FRAMEWORKID -_HEALTHCHECK_HTTP.containing_type = _HEALTHCHECK; -_HEALTHCHECK.fields_by_name['http'].message_type = _HEALTHCHECK_HTTP -_COMMANDINFO_URI.containing_type = _COMMANDINFO; -_COMMANDINFO_CONTAINERINFO.containing_type = _COMMANDINFO; -_COMMANDINFO.fields_by_name['container'].message_type = _COMMANDINFO_CONTAINERINFO -_COMMANDINFO.fields_by_name['uris'].message_type = _COMMANDINFO_URI -_COMMANDINFO.fields_by_name['environment'].message_type = _ENVIRONMENT -_COMMANDINFO.fields_by_name['health_check'].message_type = _HEALTHCHECK -_EXECUTORINFO.fields_by_name['executor_id'].message_type = _EXECUTORID -_EXECUTORINFO.fields_by_name['framework_id'].message_type = _FRAMEWORKID -_EXECUTORINFO.fields_by_name['command'].message_type = _COMMANDINFO -_EXECUTORINFO.fields_by_name['resources'].message_type = _RESOURCE -_SLAVEINFO.fields_by_name['resources'].message_type = _RESOURCE -_SLAVEINFO.fields_by_name['attributes'].message_type = _ATTRIBUTE -_SLAVEINFO.fields_by_name['id'].message_type = _SLAVEID -_VALUE_SCALAR.containing_type = _VALUE; -_VALUE_RANGE.containing_type = _VALUE; -_VALUE_RANGES.fields_by_name['range'].message_type = _VALUE_RANGE -_VALUE_RANGES.containing_type = _VALUE; -_VALUE_SET.containing_type = _VALUE; -_VALUE_TEXT.containing_type = _VALUE; -_VALUE.fields_by_name['type'].enum_type = _VALUE_TYPE -_VALUE.fields_by_name['scalar'].message_type = _VALUE_SCALAR -_VALUE.fields_by_name['ranges'].message_type = _VALUE_RANGES -_VALUE.fields_by_name['set'].message_type = _VALUE_SET -_VALUE.fields_by_name['text'].message_type = _VALUE_TEXT -_VALUE_TYPE.containing_type = _VALUE; -_ATTRIBUTE.fields_by_name['type'].enum_type = _VALUE_TYPE -_ATTRIBUTE.fields_by_name['scalar'].message_type = _VALUE_SCALAR -_ATTRIBUTE.fields_by_name['ranges'].message_type = _VALUE_RANGES -_ATTRIBUTE.fields_by_name['set'].message_type = _VALUE_SET -_ATTRIBUTE.fields_by_name['text'].message_type = _VALUE_TEXT -_RESOURCE.fields_by_name['type'].enum_type = _VALUE_TYPE -_RESOURCE.fields_by_name['scalar'].message_type = _VALUE_SCALAR -_RESOURCE.fields_by_name['ranges'].message_type = _VALUE_RANGES -_RESOURCE.fields_by_name['set'].message_type = _VALUE_SET -_RESOURCEUSAGE.fields_by_name['slave_id'].message_type = _SLAVEID -_RESOURCEUSAGE.fields_by_name['framework_id'].message_type = _FRAMEWORKID -_RESOURCEUSAGE.fields_by_name['executor_id'].message_type = _EXECUTORID -_RESOURCEUSAGE.fields_by_name['task_id'].message_type = _TASKID -_RESOURCEUSAGE.fields_by_name['statistics'].message_type = _RESOURCESTATISTICS -_REQUEST.fields_by_name['slave_id'].message_type = _SLAVEID -_REQUEST.fields_by_name['resources'].message_type = _RESOURCE -_OFFER.fields_by_name['id'].message_type = _OFFERID -_OFFER.fields_by_name['framework_id'].message_type = _FRAMEWORKID -_OFFER.fields_by_name['slave_id'].message_type = _SLAVEID -_OFFER.fields_by_name['resources'].message_type = _RESOURCE -_OFFER.fields_by_name['attributes'].message_type = _ATTRIBUTE -_OFFER.fields_by_name['executor_ids'].message_type = _EXECUTORID -_TASKINFO.fields_by_name['task_id'].message_type = _TASKID -_TASKINFO.fields_by_name['slave_id'].message_type = _SLAVEID -_TASKINFO.fields_by_name['resources'].message_type = _RESOURCE -_TASKINFO.fields_by_name['executor'].message_type = _EXECUTORINFO -_TASKINFO.fields_by_name['command'].message_type = _COMMANDINFO -_TASKSTATUS.fields_by_name['task_id'].message_type = _TASKID -_TASKSTATUS.fields_by_name['state'].enum_type = _TASKSTATE -_TASKSTATUS.fields_by_name['slave_id'].message_type = _SLAVEID -_ENVIRONMENT_VARIABLE.containing_type = _ENVIRONMENT; -_ENVIRONMENT.fields_by_name['variables'].message_type = _ENVIRONMENT_VARIABLE -_PARAMETERS.fields_by_name['parameter'].message_type = _PARAMETER -_ACL_ENTITY.fields_by_name['type'].enum_type = _ACL_ENTITY_TYPE -_ACL_ENTITY.containing_type = _ACL; -_ACL_ENTITY_TYPE.containing_type = _ACL_ENTITY; -_ACL_RUNTASKS.fields_by_name['principals'].message_type = _ACL_ENTITY -_ACL_RUNTASKS.fields_by_name['users'].message_type = _ACL_ENTITY -_ACL_RUNTASKS.containing_type = _ACL; -_ACL_RECEIVEOFFERS.fields_by_name['principals'].message_type = _ACL_ENTITY -_ACL_RECEIVEOFFERS.fields_by_name['roles'].message_type = _ACL_ENTITY -_ACL_RECEIVEOFFERS.containing_type = _ACL; -_ACL_HTTPGET.fields_by_name['usernames'].message_type = _ACL_ENTITY -_ACL_HTTPGET.fields_by_name['ips'].message_type = _ACL_ENTITY -_ACL_HTTPGET.fields_by_name['hostnames'].message_type = _ACL_ENTITY -_ACL_HTTPGET.fields_by_name['urls'].message_type = _ACL_ENTITY -_ACL_HTTPGET.containing_type = _ACL; -_ACL_HTTPPUT.fields_by_name['usernames'].message_type = _ACL_ENTITY -_ACL_HTTPPUT.fields_by_name['ips'].message_type = _ACL_ENTITY -_ACL_HTTPPUT.fields_by_name['hostnames'].message_type = _ACL_ENTITY -_ACL_HTTPPUT.fields_by_name['urls'].message_type = _ACL_ENTITY -_ACL_HTTPPUT.containing_type = _ACL; -_ACLS.fields_by_name['run_tasks'].message_type = _ACL_RUNTASKS -_ACLS.fields_by_name['receive_offers'].message_type = _ACL_RECEIVEOFFERS -_ACLS.fields_by_name['http_get'].message_type = _ACL_HTTPGET -_ACLS.fields_by_name['http_put'].message_type = _ACL_HTTPPUT -DESCRIPTOR.message_types_by_name['FrameworkID'] = _FRAMEWORKID -DESCRIPTOR.message_types_by_name['OfferID'] = _OFFERID -DESCRIPTOR.message_types_by_name['SlaveID'] = _SLAVEID -DESCRIPTOR.message_types_by_name['TaskID'] = _TASKID -DESCRIPTOR.message_types_by_name['ExecutorID'] = _EXECUTORID -DESCRIPTOR.message_types_by_name['ContainerID'] = _CONTAINERID -DESCRIPTOR.message_types_by_name['FrameworkInfo'] = _FRAMEWORKINFO -DESCRIPTOR.message_types_by_name['HealthCheck'] = _HEALTHCHECK -DESCRIPTOR.message_types_by_name['CommandInfo'] = _COMMANDINFO -DESCRIPTOR.message_types_by_name['ExecutorInfo'] = _EXECUTORINFO -DESCRIPTOR.message_types_by_name['MasterInfo'] = _MASTERINFO -DESCRIPTOR.message_types_by_name['SlaveInfo'] = _SLAVEINFO -DESCRIPTOR.message_types_by_name['Value'] = _VALUE -DESCRIPTOR.message_types_by_name['Attribute'] = _ATTRIBUTE -DESCRIPTOR.message_types_by_name['Resource'] = _RESOURCE -DESCRIPTOR.message_types_by_name['ResourceStatistics'] = _RESOURCESTATISTICS -DESCRIPTOR.message_types_by_name['ResourceUsage'] = _RESOURCEUSAGE -DESCRIPTOR.message_types_by_name['Request'] = _REQUEST -DESCRIPTOR.message_types_by_name['Offer'] = _OFFER -DESCRIPTOR.message_types_by_name['TaskInfo'] = _TASKINFO -DESCRIPTOR.message_types_by_name['TaskStatus'] = _TASKSTATUS -DESCRIPTOR.message_types_by_name['Filters'] = _FILTERS -DESCRIPTOR.message_types_by_name['Environment'] = _ENVIRONMENT -DESCRIPTOR.message_types_by_name['Parameter'] = _PARAMETER -DESCRIPTOR.message_types_by_name['Parameters'] = _PARAMETERS -DESCRIPTOR.message_types_by_name['Credential'] = _CREDENTIAL -DESCRIPTOR.message_types_by_name['ACL'] = _ACL -DESCRIPTOR.message_types_by_name['ACLs'] = _ACLS - -class FrameworkID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKID - - # @@protoc_insertion_point(class_scope:mesos.FrameworkID) - -class OfferID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _OFFERID - - # @@protoc_insertion_point(class_scope:mesos.OfferID) - -class SlaveID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SLAVEID - - # @@protoc_insertion_point(class_scope:mesos.SlaveID) - -class TaskID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _TASKID - - # @@protoc_insertion_point(class_scope:mesos.TaskID) - -class ExecutorID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXECUTORID - - # @@protoc_insertion_point(class_scope:mesos.ExecutorID) - -class ContainerID(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _CONTAINERID - - # @@protoc_insertion_point(class_scope:mesos.ContainerID) - -class FrameworkInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKINFO - - # @@protoc_insertion_point(class_scope:mesos.FrameworkInfo) - -class HealthCheck(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class HTTP(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _HEALTHCHECK_HTTP - - # @@protoc_insertion_point(class_scope:mesos.HealthCheck.HTTP) - DESCRIPTOR = _HEALTHCHECK - - # @@protoc_insertion_point(class_scope:mesos.HealthCheck) - -class CommandInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class URI(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _COMMANDINFO_URI - - # @@protoc_insertion_point(class_scope:mesos.CommandInfo.URI) - - class ContainerInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _COMMANDINFO_CONTAINERINFO - - # @@protoc_insertion_point(class_scope:mesos.CommandInfo.ContainerInfo) - DESCRIPTOR = _COMMANDINFO - - # @@protoc_insertion_point(class_scope:mesos.CommandInfo) - -class ExecutorInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXECUTORINFO - - # @@protoc_insertion_point(class_scope:mesos.ExecutorInfo) - -class MasterInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _MASTERINFO - - # @@protoc_insertion_point(class_scope:mesos.MasterInfo) - -class SlaveInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SLAVEINFO - - # @@protoc_insertion_point(class_scope:mesos.SlaveInfo) - -class Value(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class Scalar(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _VALUE_SCALAR - - # @@protoc_insertion_point(class_scope:mesos.Value.Scalar) - - class Range(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _VALUE_RANGE - - # @@protoc_insertion_point(class_scope:mesos.Value.Range) - - class Ranges(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _VALUE_RANGES - - # @@protoc_insertion_point(class_scope:mesos.Value.Ranges) - - class Set(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _VALUE_SET - - # @@protoc_insertion_point(class_scope:mesos.Value.Set) - - class Text(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _VALUE_TEXT - - # @@protoc_insertion_point(class_scope:mesos.Value.Text) - DESCRIPTOR = _VALUE - - # @@protoc_insertion_point(class_scope:mesos.Value) - -class Attribute(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ATTRIBUTE - - # @@protoc_insertion_point(class_scope:mesos.Attribute) - -class Resource(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESOURCE - - # @@protoc_insertion_point(class_scope:mesos.Resource) - -class ResourceStatistics(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESOURCESTATISTICS - - # @@protoc_insertion_point(class_scope:mesos.ResourceStatistics) - -class ResourceUsage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESOURCEUSAGE - - # @@protoc_insertion_point(class_scope:mesos.ResourceUsage) - -class Request(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REQUEST - - # @@protoc_insertion_point(class_scope:mesos.Request) - -class Offer(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _OFFER - - # @@protoc_insertion_point(class_scope:mesos.Offer) - -class TaskInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _TASKINFO - - # @@protoc_insertion_point(class_scope:mesos.TaskInfo) - -class TaskStatus(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _TASKSTATUS - - # @@protoc_insertion_point(class_scope:mesos.TaskStatus) - -class Filters(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FILTERS - - # @@protoc_insertion_point(class_scope:mesos.Filters) - -class Environment(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class Variable(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ENVIRONMENT_VARIABLE - - # @@protoc_insertion_point(class_scope:mesos.Environment.Variable) - DESCRIPTOR = _ENVIRONMENT - - # @@protoc_insertion_point(class_scope:mesos.Environment) - -class Parameter(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _PARAMETER - - # @@protoc_insertion_point(class_scope:mesos.Parameter) - -class Parameters(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _PARAMETERS - - # @@protoc_insertion_point(class_scope:mesos.Parameters) - -class Credential(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _CREDENTIAL - - # @@protoc_insertion_point(class_scope:mesos.Credential) - -class ACL(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class Entity(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACL_ENTITY - - # @@protoc_insertion_point(class_scope:mesos.ACL.Entity) - - class RunTasks(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACL_RUNTASKS - - # @@protoc_insertion_point(class_scope:mesos.ACL.RunTasks) - - class ReceiveOffers(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACL_RECEIVEOFFERS - - # @@protoc_insertion_point(class_scope:mesos.ACL.ReceiveOffers) - - class HTTPGet(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACL_HTTPGET - - # @@protoc_insertion_point(class_scope:mesos.ACL.HTTPGet) - - class HTTPPut(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACL_HTTPPUT - - # @@protoc_insertion_point(class_scope:mesos.ACL.HTTPPut) - DESCRIPTOR = _ACL - - # @@protoc_insertion_point(class_scope:mesos.ACL) - -class ACLs(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ACLS - - # @@protoc_insertion_point(class_scope:mesos.ACLs) - -# @@protoc_insertion_point(module_scope) diff --git a/dpark/pymesos/messages.proto b/dpark/pymesos/messages.proto deleted file mode 100644 index 6f6e570b..00000000 --- a/dpark/pymesos/messages.proto +++ /dev/null @@ -1,387 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import "mesos.proto"; - -package mesos.internal; - -// TODO(benh): Provide comments for each of these messages. Also, -// consider splitting these messages into different "packages" which -// represent which messages get handled by which components (e.g., the -// "mesos.internal.executor" package includes messages that the -// executor handles). - - -// TODO(benh): It would be great if this could just be a -// TaskInfo wherever it gets used! However, doing so would -// require adding the framework_id field, the executor_id field, and -// the state field into TaskInfo though (or send them another -// way). Also, one performance reason why we don't do that now is -// because storing whatever data is coupled with a TaskInfo -// could be large and unnecessary. -// TODO(bmahler): Add executor_uuid here, and send it to the master. This will -// allow us to expose executor work directories for tasks in the webui when -// looking from the master level. Currently only the slave knows which run the -// task belongs to. -message Task { - required string name = 1; - required TaskID task_id = 2; - required FrameworkID framework_id = 3; - optional ExecutorID executor_id = 4; - required SlaveID slave_id = 5; - required TaskState state = 6; - repeated Resource resources = 7; - repeated TaskStatus statuses = 8; -} - -// Describes a role, which are used to group frameworks for allocation -// decisions, depending on the allocation policy being used. -// The weight field can be used to indicate forms of priority. -message RoleInfo { - required string name = 1; - optional double weight = 2 [default = 1]; -} - - -message StatusUpdate { - required FrameworkID framework_id = 1; - optional ExecutorID executor_id = 2; - optional SlaveID slave_id = 3; - required TaskStatus status = 4; - required double timestamp = 5; - required bytes uuid = 6; -} - - -// This message encapsulates how we checkpoint a status update to disk. -// NOTE: If type == UPDATE, the 'update' field is required. -// NOTE: If type == ACK, the 'uuid' field is required. -message StatusUpdateRecord { - enum Type { - UPDATE = 0; - ACK = 1; - } - required Type type = 1; - optional StatusUpdate update = 2; - optional bytes uuid = 3; -} - - -message SubmitSchedulerRequest -{ - required string name = 1; -} - - -message SubmitSchedulerResponse -{ - required bool okay = 1; -} - - -message ExecutorToFrameworkMessage { - required SlaveID slave_id = 1; - required FrameworkID framework_id = 2; - required ExecutorID executor_id = 3; - required bytes data = 4; -} - - -message FrameworkToExecutorMessage { - required SlaveID slave_id = 1; - required FrameworkID framework_id = 2; - required ExecutorID executor_id = 3; - required bytes data = 4; -} - - -message RegisterFrameworkMessage { - required FrameworkInfo framework = 1; -} - - -message ReregisterFrameworkMessage { - required FrameworkInfo framework = 2; - required bool failover = 3; -} - - -message FrameworkRegisteredMessage { - required FrameworkID framework_id = 1; - required MasterInfo master_info = 2; -} - -message FrameworkReregisteredMessage { - required FrameworkID framework_id = 1; - required MasterInfo master_info = 2; -} - -message UnregisterFrameworkMessage { - required FrameworkID framework_id = 1; -} - - -message DeactivateFrameworkMessage { - required FrameworkID framework_id = 1; -} - - -message ResourceRequestMessage { - required FrameworkID framework_id = 1; - repeated Request requests = 2; -} - - -message ResourceOffersMessage { - repeated Offer offers = 1; - repeated string pids = 2; -} - - -message LaunchTasksMessage { - required FrameworkID framework_id = 1; - optional OfferID offer_id = 2; // Deprecated. - repeated TaskInfo tasks = 3; - required Filters filters = 5; - repeated OfferID offer_ids = 6; -} - - -message RescindResourceOfferMessage { - required OfferID offer_id = 1; -} - - -message ReviveOffersMessage { - required FrameworkID framework_id = 1; -} - - -message RunTaskMessage { - required FrameworkID framework_id = 1; - required FrameworkInfo framework = 2; - required string pid = 3; - required TaskInfo task = 4; -} - - -message KillTaskMessage { - // TODO(bmahler): Include the SlaveID here to improve the Master's - // ability to respond for non-activated slaves. - required FrameworkID framework_id = 1; - required TaskID task_id = 2; -} - - -// NOTE: If 'pid' is present, scheduler driver sends an -// acknowledgement to the pid. -message StatusUpdateMessage { - required StatusUpdate update = 1; - optional string pid = 2; -} - - -message StatusUpdateAcknowledgementMessage { - required SlaveID slave_id = 1; - required FrameworkID framework_id = 2; - required TaskID task_id = 3; - required bytes uuid = 4; -} - - -message LostSlaveMessage { - required SlaveID slave_id = 1; -} - - -message ReconcileTasksMessage { - required FrameworkID framework_id = 1; - repeated TaskStatus statuses = 2; -} - - -message FrameworkErrorMessage { - required string message = 2; -} - - -message RegisterSlaveMessage { - required SlaveInfo slave = 1; -} - - -message ReregisterSlaveMessage { - // TODO(bmahler): Deprecate and remove the explicit slave_id as - // SlaveInfo already includes this information. - required SlaveID slave_id = 1; - required SlaveInfo slave = 2; - repeated ExecutorInfo executor_infos = 4; - repeated Task tasks = 3; - repeated Archive.Framework completed_frameworks = 5; -} - - -message SlaveRegisteredMessage { - required SlaveID slave_id = 1; -} - - -message SlaveReregisteredMessage { - required SlaveID slave_id = 1; -} - - -message UnregisterSlaveMessage { - required SlaveID slave_id = 1; -} - - -message HeartbeatMessage { - required SlaveID slave_id = 1; -} - - -// Tells a slave to shut down all executors of the given framework. -message ShutdownFrameworkMessage { - required FrameworkID framework_id = 1; -} - - -// Tells the executor to initiate a shut down by invoking -// Executor::shutdown. -message ShutdownExecutorMessage {} - - -message UpdateFrameworkMessage { - required FrameworkID framework_id = 1; - required string pid = 2; -} - - -message RegisterExecutorMessage { - required FrameworkID framework_id = 1; - required ExecutorID executor_id = 2; -} - - -message ExecutorRegisteredMessage { - required ExecutorInfo executor_info = 2; - required FrameworkID framework_id = 3; - required FrameworkInfo framework_info = 4; - required SlaveID slave_id = 5; - required SlaveInfo slave_info = 6; -} - - -message ExecutorReregisteredMessage { - required SlaveID slave_id = 1; - required SlaveInfo slave_info = 2; -} - - -message ExitedExecutorMessage { - required SlaveID slave_id = 1; - required FrameworkID framework_id = 2; - required ExecutorID executor_id = 3; - required int32 status = 4; -} - - -message ReconnectExecutorMessage { - required SlaveID slave_id = 1; -} - - -message ReregisterExecutorMessage { - required ExecutorID executor_id = 1; - required FrameworkID framework_id = 2; - repeated TaskInfo tasks = 3; - repeated StatusUpdate updates = 4; -} - - -message RegisterProjdMessage { - required string project = 1; -} - - -message ProjdReadyMessage { - required string project = 1; -} - - -message ProjdUpdateResourcesMessage { - optional Parameters parameters = 1; -} - - -message FrameworkExpiredMessage { - required FrameworkID framework_id = 1; -} - - -message ShutdownMessage { - optional string message = 1; -} - - -message AuthenticateMessage { - required string pid = 1; // PID that needs to be authenticated. -} - - -message AuthenticationMechanismsMessage { - repeated string mechanisms = 1; // List of available SASL mechanisms. -} - - -message AuthenticationStartMessage { - required string mechanism = 1; - optional string data = 2; -} - - -message AuthenticationStepMessage { - required bytes data = 1; -} - - -message AuthenticationCompletedMessage {} - - -message AuthenticationFailedMessage {} - - -message AuthenticationErrorMessage { - optional string error = 1; -} - - -// TODO(adam-mesos): Move this to an 'archive' package. -/** - * Describes Completed Frameworks, etc. for archival. - */ -message Archive { - message Framework { - required FrameworkInfo framework_info = 1; - optional string pid = 2; - repeated Task tasks = 3; - } - repeated Framework frameworks = 1; -} - diff --git a/dpark/pymesos/messages_pb2.py b/dpark/pymesos/messages_pb2.py deleted file mode 100644 index 544f7210..00000000 --- a/dpark/pymesos/messages_pb2.py +++ /dev/null @@ -1,2442 +0,0 @@ -# Generated by the protocol buffer compiler. DO NOT EDIT! - -from google.protobuf import descriptor -from google.protobuf import message -from google.protobuf import reflection -from google.protobuf import descriptor_pb2 -# @@protoc_insertion_point(imports) - - -import mesos_pb2 - -DESCRIPTOR = descriptor.FileDescriptor( - name='messages.proto', - package='mesos.internal', - serialized_pb='\n\x0emessages.proto\x12\x0emesos.internal\x1a\x0bmesos.proto\"\x92\x02\n\x04Task\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x1e\n\x07task_id\x18\x02 \x02(\x0b\x32\r.mesos.TaskID\x12(\n\x0c\x66ramework_id\x18\x03 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x04 \x01(\x0b\x32\x11.mesos.ExecutorID\x12 \n\x08slave_id\x18\x05 \x02(\x0b\x32\x0e.mesos.SlaveID\x12\x1f\n\x05state\x18\x06 \x02(\x0e\x32\x10.mesos.TaskState\x12\"\n\tresources\x18\x07 \x03(\x0b\x32\x0f.mesos.Resource\x12#\n\x08statuses\x18\x08 \x03(\x0b\x32\x11.mesos.TaskStatus\"+\n\x08RoleInfo\x12\x0c\n\x04name\x18\x01 \x02(\t\x12\x11\n\x06weight\x18\x02 \x01(\x01:\x01\x31\"\xc6\x01\n\x0cStatusUpdate\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x02 \x01(\x0b\x32\x11.mesos.ExecutorID\x12 \n\x08slave_id\x18\x03 \x01(\x0b\x32\x0e.mesos.SlaveID\x12!\n\x06status\x18\x04 \x02(\x0b\x32\x11.mesos.TaskStatus\x12\x11\n\ttimestamp\x18\x05 \x02(\x01\x12\x0c\n\x04uuid\x18\x06 \x02(\x0c\"\xa4\x01\n\x12StatusUpdateRecord\x12\x35\n\x04type\x18\x01 \x02(\x0e\x32\'.mesos.internal.StatusUpdateRecord.Type\x12,\n\x06update\x18\x02 \x01(\x0b\x32\x1c.mesos.internal.StatusUpdate\x12\x0c\n\x04uuid\x18\x03 \x01(\x0c\"\x1b\n\x04Type\x12\n\n\x06UPDATE\x10\x00\x12\x07\n\x03\x41\x43K\x10\x01\"&\n\x16SubmitSchedulerRequest\x12\x0c\n\x04name\x18\x01 \x02(\t\"\'\n\x17SubmitSchedulerResponse\x12\x0c\n\x04okay\x18\x01 \x02(\x08\"\x9e\x01\n\x1a\x45xecutorToFrameworkMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x03 \x02(\x0b\x32\x11.mesos.ExecutorID\x12\x0c\n\x04\x64\x61ta\x18\x04 \x02(\x0c\"\x9e\x01\n\x1a\x46rameworkToExecutorMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x03 \x02(\x0b\x32\x11.mesos.ExecutorID\x12\x0c\n\x04\x64\x61ta\x18\x04 \x02(\x0c\"C\n\x18RegisterFrameworkMessage\x12\'\n\tframework\x18\x01 \x02(\x0b\x32\x14.mesos.FrameworkInfo\"W\n\x1aReregisterFrameworkMessage\x12\'\n\tframework\x18\x02 \x02(\x0b\x32\x14.mesos.FrameworkInfo\x12\x10\n\x08\x66\x61ilover\x18\x03 \x02(\x08\"n\n\x1a\x46rameworkRegisteredMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0bmaster_info\x18\x02 \x02(\x0b\x32\x11.mesos.MasterInfo\"p\n\x1c\x46rameworkReregisteredMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0bmaster_info\x18\x02 \x02(\x0b\x32\x11.mesos.MasterInfo\"F\n\x1aUnregisterFrameworkMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\"F\n\x1a\x44\x65\x61\x63tivateFrameworkMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\"d\n\x16ResourceRequestMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12 \n\x08requests\x18\x02 \x03(\x0b\x32\x0e.mesos.Request\"C\n\x15ResourceOffersMessage\x12\x1c\n\x06offers\x18\x01 \x03(\x0b\x32\x0c.mesos.Offer\x12\x0c\n\x04pids\x18\x02 \x03(\t\"\xc4\x01\n\x12LaunchTasksMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12 \n\x08offer_id\x18\x02 \x01(\x0b\x32\x0e.mesos.OfferID\x12\x1e\n\x05tasks\x18\x03 \x03(\x0b\x32\x0f.mesos.TaskInfo\x12\x1f\n\x07\x66ilters\x18\x05 \x02(\x0b\x32\x0e.mesos.Filters\x12!\n\toffer_ids\x18\x06 \x03(\x0b\x32\x0e.mesos.OfferID\"?\n\x1bRescindResourceOfferMessage\x12 \n\x08offer_id\x18\x01 \x02(\x0b\x32\x0e.mesos.OfferID\"?\n\x13ReviveOffersMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\"\x8f\x01\n\x0eRunTaskMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12\'\n\tframework\x18\x02 \x02(\x0b\x32\x14.mesos.FrameworkInfo\x12\x0b\n\x03pid\x18\x03 \x02(\t\x12\x1d\n\x04task\x18\x04 \x02(\x0b\x32\x0f.mesos.TaskInfo\"[\n\x0fKillTaskMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12\x1e\n\x07task_id\x18\x02 \x02(\x0b\x32\r.mesos.TaskID\"P\n\x13StatusUpdateMessage\x12,\n\x06update\x18\x01 \x02(\x0b\x32\x1c.mesos.internal.StatusUpdate\x12\x0b\n\x03pid\x18\x02 \x01(\t\"\x9e\x01\n\"StatusUpdateAcknowledgementMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12\x1e\n\x07task_id\x18\x03 \x02(\x0b\x32\r.mesos.TaskID\x12\x0c\n\x04uuid\x18\x04 \x02(\x0c\"4\n\x10LostSlaveMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\"f\n\x15ReconcileTasksMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12#\n\x08statuses\x18\x02 \x03(\x0b\x32\x11.mesos.TaskStatus\"(\n\x15\x46rameworkErrorMessage\x12\x0f\n\x07message\x18\x02 \x02(\t\"7\n\x14RegisterSlaveMessage\x12\x1f\n\x05slave\x18\x01 \x02(\x0b\x32\x10.mesos.SlaveInfo\"\xee\x01\n\x16ReregisterSlaveMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12\x1f\n\x05slave\x18\x02 \x02(\x0b\x32\x10.mesos.SlaveInfo\x12+\n\x0e\x65xecutor_infos\x18\x04 \x03(\x0b\x32\x13.mesos.ExecutorInfo\x12#\n\x05tasks\x18\x03 \x03(\x0b\x32\x14.mesos.internal.Task\x12?\n\x14\x63ompleted_frameworks\x18\x05 \x03(\x0b\x32!.mesos.internal.Archive.Framework\":\n\x16SlaveRegisteredMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\"<\n\x18SlaveReregisteredMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\":\n\x16UnregisterSlaveMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\"4\n\x10HeartbeatMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\"D\n\x18ShutdownFrameworkMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\"\x19\n\x17ShutdownExecutorMessage\"O\n\x16UpdateFrameworkMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12\x0b\n\x03pid\x18\x02 \x02(\t\"k\n\x17RegisterExecutorMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x02 \x02(\x0b\x32\x11.mesos.ExecutorID\"\xe7\x01\n\x19\x45xecutorRegisteredMessage\x12*\n\rexecutor_info\x18\x02 \x02(\x0b\x32\x13.mesos.ExecutorInfo\x12(\n\x0c\x66ramework_id\x18\x03 \x02(\x0b\x32\x12.mesos.FrameworkID\x12,\n\x0e\x66ramework_info\x18\x04 \x02(\x0b\x32\x14.mesos.FrameworkInfo\x12 \n\x08slave_id\x18\x05 \x02(\x0b\x32\x0e.mesos.SlaveID\x12$\n\nslave_info\x18\x06 \x02(\x0b\x32\x10.mesos.SlaveInfo\"e\n\x1b\x45xecutorReregisteredMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12$\n\nslave_info\x18\x02 \x02(\x0b\x32\x10.mesos.SlaveInfo\"\x9b\x01\n\x15\x45xitedExecutorMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12&\n\x0b\x65xecutor_id\x18\x03 \x02(\x0b\x32\x11.mesos.ExecutorID\x12\x0e\n\x06status\x18\x04 \x02(\x05\"<\n\x18ReconnectExecutorMessage\x12 \n\x08slave_id\x18\x01 \x02(\x0b\x32\x0e.mesos.SlaveID\"\xbc\x01\n\x19ReregisterExecutorMessage\x12&\n\x0b\x65xecutor_id\x18\x01 \x02(\x0b\x32\x11.mesos.ExecutorID\x12(\n\x0c\x66ramework_id\x18\x02 \x02(\x0b\x32\x12.mesos.FrameworkID\x12\x1e\n\x05tasks\x18\x03 \x03(\x0b\x32\x0f.mesos.TaskInfo\x12-\n\x07updates\x18\x04 \x03(\x0b\x32\x1c.mesos.internal.StatusUpdate\"\'\n\x14RegisterProjdMessage\x12\x0f\n\x07project\x18\x01 \x02(\t\"$\n\x11ProjdReadyMessage\x12\x0f\n\x07project\x18\x01 \x02(\t\"D\n\x1bProjdUpdateResourcesMessage\x12%\n\nparameters\x18\x01 \x01(\x0b\x32\x11.mesos.Parameters\"C\n\x17\x46rameworkExpiredMessage\x12(\n\x0c\x66ramework_id\x18\x01 \x02(\x0b\x32\x12.mesos.FrameworkID\"\"\n\x0fShutdownMessage\x12\x0f\n\x07message\x18\x01 \x01(\t\"\"\n\x13\x41uthenticateMessage\x12\x0b\n\x03pid\x18\x01 \x02(\t\"5\n\x1f\x41uthenticationMechanismsMessage\x12\x12\n\nmechanisms\x18\x01 \x03(\t\"=\n\x1a\x41uthenticationStartMessage\x12\x11\n\tmechanism\x18\x01 \x02(\t\x12\x0c\n\x04\x64\x61ta\x18\x02 \x01(\t\")\n\x19\x41uthenticationStepMessage\x12\x0c\n\x04\x64\x61ta\x18\x01 \x02(\x0c\" \n\x1e\x41uthenticationCompletedMessage\"\x1d\n\x1b\x41uthenticationFailedMessage\"+\n\x1a\x41uthenticationErrorMessage\x12\r\n\x05\x65rror\x18\x01 \x01(\t\"\xad\x01\n\x07\x41rchive\x12\x35\n\nframeworks\x18\x01 \x03(\x0b\x32!.mesos.internal.Archive.Framework\x1ak\n\tFramework\x12,\n\x0e\x66ramework_info\x18\x01 \x02(\x0b\x32\x14.mesos.FrameworkInfo\x12\x0b\n\x03pid\x18\x02 \x01(\t\x12#\n\x05tasks\x18\x03 \x03(\x0b\x32\x14.mesos.internal.Task') - - - -_STATUSUPDATERECORD_TYPE = descriptor.EnumDescriptor( - name='Type', - full_name='mesos.internal.StatusUpdateRecord.Type', - filename=None, - file=DESCRIPTOR, - values=[ - descriptor.EnumValueDescriptor( - name='UPDATE', index=0, number=0, - options=None, - type=None), - descriptor.EnumValueDescriptor( - name='ACK', index=1, number=1, - options=None, - type=None), - ], - containing_type=None, - options=None, - serialized_start=708, - serialized_end=735, -) - - -_TASK = descriptor.Descriptor( - name='Task', - full_name='mesos.internal.Task', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.internal.Task.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.internal.Task.task_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.Task.framework_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.Task.executor_id', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.Task.slave_id', index=4, - number=5, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='state', full_name='mesos.internal.Task.state', index=5, - number=6, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=6, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='resources', full_name='mesos.internal.Task.resources', index=6, - number=7, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='statuses', full_name='mesos.internal.Task.statuses', index=7, - number=8, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=48, - serialized_end=322, -) - - -_ROLEINFO = descriptor.Descriptor( - name='RoleInfo', - full_name='mesos.internal.RoleInfo', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.internal.RoleInfo.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='weight', full_name='mesos.internal.RoleInfo.weight', index=1, - number=2, type=1, cpp_type=5, label=1, - has_default_value=True, default_value=1, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=324, - serialized_end=367, -) - - -_STATUSUPDATE = descriptor.Descriptor( - name='StatusUpdate', - full_name='mesos.internal.StatusUpdate', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.StatusUpdate.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.StatusUpdate.executor_id', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.StatusUpdate.slave_id', index=2, - number=3, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='status', full_name='mesos.internal.StatusUpdate.status', index=3, - number=4, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='timestamp', full_name='mesos.internal.StatusUpdate.timestamp', index=4, - number=5, type=1, cpp_type=5, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='uuid', full_name='mesos.internal.StatusUpdate.uuid', index=5, - number=6, type=12, cpp_type=9, label=2, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=370, - serialized_end=568, -) - - -_STATUSUPDATERECORD = descriptor.Descriptor( - name='StatusUpdateRecord', - full_name='mesos.internal.StatusUpdateRecord', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='type', full_name='mesos.internal.StatusUpdateRecord.type', index=0, - number=1, type=14, cpp_type=8, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='update', full_name='mesos.internal.StatusUpdateRecord.update', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='uuid', full_name='mesos.internal.StatusUpdateRecord.uuid', index=2, - number=3, type=12, cpp_type=9, label=1, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - _STATUSUPDATERECORD_TYPE, - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=571, - serialized_end=735, -) - - -_SUBMITSCHEDULERREQUEST = descriptor.Descriptor( - name='SubmitSchedulerRequest', - full_name='mesos.internal.SubmitSchedulerRequest', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='name', full_name='mesos.internal.SubmitSchedulerRequest.name', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=737, - serialized_end=775, -) - - -_SUBMITSCHEDULERRESPONSE = descriptor.Descriptor( - name='SubmitSchedulerResponse', - full_name='mesos.internal.SubmitSchedulerResponse', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='okay', full_name='mesos.internal.SubmitSchedulerResponse.okay', index=0, - number=1, type=8, cpp_type=7, label=2, - has_default_value=False, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=777, - serialized_end=816, -) - - -_EXECUTORTOFRAMEWORKMESSAGE = descriptor.Descriptor( - name='ExecutorToFrameworkMessage', - full_name='mesos.internal.ExecutorToFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ExecutorToFrameworkMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ExecutorToFrameworkMessage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.ExecutorToFrameworkMessage.executor_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.internal.ExecutorToFrameworkMessage.data', index=3, - number=4, type=12, cpp_type=9, label=2, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=819, - serialized_end=977, -) - - -_FRAMEWORKTOEXECUTORMESSAGE = descriptor.Descriptor( - name='FrameworkToExecutorMessage', - full_name='mesos.internal.FrameworkToExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.FrameworkToExecutorMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.FrameworkToExecutorMessage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.FrameworkToExecutorMessage.executor_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.internal.FrameworkToExecutorMessage.data', index=3, - number=4, type=12, cpp_type=9, label=2, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=980, - serialized_end=1138, -) - - -_REGISTERFRAMEWORKMESSAGE = descriptor.Descriptor( - name='RegisterFrameworkMessage', - full_name='mesos.internal.RegisterFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework', full_name='mesos.internal.RegisterFrameworkMessage.framework', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1140, - serialized_end=1207, -) - - -_REREGISTERFRAMEWORKMESSAGE = descriptor.Descriptor( - name='ReregisterFrameworkMessage', - full_name='mesos.internal.ReregisterFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework', full_name='mesos.internal.ReregisterFrameworkMessage.framework', index=0, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='failover', full_name='mesos.internal.ReregisterFrameworkMessage.failover', index=1, - number=3, type=8, cpp_type=7, label=2, - has_default_value=False, default_value=False, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1209, - serialized_end=1296, -) - - -_FRAMEWORKREGISTEREDMESSAGE = descriptor.Descriptor( - name='FrameworkRegisteredMessage', - full_name='mesos.internal.FrameworkRegisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.FrameworkRegisteredMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='master_info', full_name='mesos.internal.FrameworkRegisteredMessage.master_info', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1298, - serialized_end=1408, -) - - -_FRAMEWORKREREGISTEREDMESSAGE = descriptor.Descriptor( - name='FrameworkReregisteredMessage', - full_name='mesos.internal.FrameworkReregisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.FrameworkReregisteredMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='master_info', full_name='mesos.internal.FrameworkReregisteredMessage.master_info', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1410, - serialized_end=1522, -) - - -_UNREGISTERFRAMEWORKMESSAGE = descriptor.Descriptor( - name='UnregisterFrameworkMessage', - full_name='mesos.internal.UnregisterFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.UnregisterFrameworkMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1524, - serialized_end=1594, -) - - -_DEACTIVATEFRAMEWORKMESSAGE = descriptor.Descriptor( - name='DeactivateFrameworkMessage', - full_name='mesos.internal.DeactivateFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.DeactivateFrameworkMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1596, - serialized_end=1666, -) - - -_RESOURCEREQUESTMESSAGE = descriptor.Descriptor( - name='ResourceRequestMessage', - full_name='mesos.internal.ResourceRequestMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ResourceRequestMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='requests', full_name='mesos.internal.ResourceRequestMessage.requests', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1668, - serialized_end=1768, -) - - -_RESOURCEOFFERSMESSAGE = descriptor.Descriptor( - name='ResourceOffersMessage', - full_name='mesos.internal.ResourceOffersMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='offers', full_name='mesos.internal.ResourceOffersMessage.offers', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pids', full_name='mesos.internal.ResourceOffersMessage.pids', index=1, - number=2, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1770, - serialized_end=1837, -) - - -_LAUNCHTASKSMESSAGE = descriptor.Descriptor( - name='LaunchTasksMessage', - full_name='mesos.internal.LaunchTasksMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.LaunchTasksMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='offer_id', full_name='mesos.internal.LaunchTasksMessage.offer_id', index=1, - number=2, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='tasks', full_name='mesos.internal.LaunchTasksMessage.tasks', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='filters', full_name='mesos.internal.LaunchTasksMessage.filters', index=3, - number=5, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='offer_ids', full_name='mesos.internal.LaunchTasksMessage.offer_ids', index=4, - number=6, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=1840, - serialized_end=2036, -) - - -_RESCINDRESOURCEOFFERMESSAGE = descriptor.Descriptor( - name='RescindResourceOfferMessage', - full_name='mesos.internal.RescindResourceOfferMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='offer_id', full_name='mesos.internal.RescindResourceOfferMessage.offer_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2038, - serialized_end=2101, -) - - -_REVIVEOFFERSMESSAGE = descriptor.Descriptor( - name='ReviveOffersMessage', - full_name='mesos.internal.ReviveOffersMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ReviveOffersMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2103, - serialized_end=2166, -) - - -_RUNTASKMESSAGE = descriptor.Descriptor( - name='RunTaskMessage', - full_name='mesos.internal.RunTaskMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.RunTaskMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework', full_name='mesos.internal.RunTaskMessage.framework', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pid', full_name='mesos.internal.RunTaskMessage.pid', index=2, - number=3, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task', full_name='mesos.internal.RunTaskMessage.task', index=3, - number=4, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2169, - serialized_end=2312, -) - - -_KILLTASKMESSAGE = descriptor.Descriptor( - name='KillTaskMessage', - full_name='mesos.internal.KillTaskMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.KillTaskMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.internal.KillTaskMessage.task_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2314, - serialized_end=2405, -) - - -_STATUSUPDATEMESSAGE = descriptor.Descriptor( - name='StatusUpdateMessage', - full_name='mesos.internal.StatusUpdateMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='update', full_name='mesos.internal.StatusUpdateMessage.update', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pid', full_name='mesos.internal.StatusUpdateMessage.pid', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2407, - serialized_end=2487, -) - - -_STATUSUPDATEACKNOWLEDGEMENTMESSAGE = descriptor.Descriptor( - name='StatusUpdateAcknowledgementMessage', - full_name='mesos.internal.StatusUpdateAcknowledgementMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.StatusUpdateAcknowledgementMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.StatusUpdateAcknowledgementMessage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='task_id', full_name='mesos.internal.StatusUpdateAcknowledgementMessage.task_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='uuid', full_name='mesos.internal.StatusUpdateAcknowledgementMessage.uuid', index=3, - number=4, type=12, cpp_type=9, label=2, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2490, - serialized_end=2648, -) - - -_LOSTSLAVEMESSAGE = descriptor.Descriptor( - name='LostSlaveMessage', - full_name='mesos.internal.LostSlaveMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.LostSlaveMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2650, - serialized_end=2702, -) - - -_RECONCILETASKSMESSAGE = descriptor.Descriptor( - name='ReconcileTasksMessage', - full_name='mesos.internal.ReconcileTasksMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ReconcileTasksMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='statuses', full_name='mesos.internal.ReconcileTasksMessage.statuses', index=1, - number=2, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2704, - serialized_end=2806, -) - - -_FRAMEWORKERRORMESSAGE = descriptor.Descriptor( - name='FrameworkErrorMessage', - full_name='mesos.internal.FrameworkErrorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='message', full_name='mesos.internal.FrameworkErrorMessage.message', index=0, - number=2, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2808, - serialized_end=2848, -) - - -_REGISTERSLAVEMESSAGE = descriptor.Descriptor( - name='RegisterSlaveMessage', - full_name='mesos.internal.RegisterSlaveMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave', full_name='mesos.internal.RegisterSlaveMessage.slave', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2850, - serialized_end=2905, -) - - -_REREGISTERSLAVEMESSAGE = descriptor.Descriptor( - name='ReregisterSlaveMessage', - full_name='mesos.internal.ReregisterSlaveMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ReregisterSlaveMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave', full_name='mesos.internal.ReregisterSlaveMessage.slave', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_infos', full_name='mesos.internal.ReregisterSlaveMessage.executor_infos', index=2, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='tasks', full_name='mesos.internal.ReregisterSlaveMessage.tasks', index=3, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='completed_frameworks', full_name='mesos.internal.ReregisterSlaveMessage.completed_frameworks', index=4, - number=5, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=2908, - serialized_end=3146, -) - - -_SLAVEREGISTEREDMESSAGE = descriptor.Descriptor( - name='SlaveRegisteredMessage', - full_name='mesos.internal.SlaveRegisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.SlaveRegisteredMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3148, - serialized_end=3206, -) - - -_SLAVEREREGISTEREDMESSAGE = descriptor.Descriptor( - name='SlaveReregisteredMessage', - full_name='mesos.internal.SlaveReregisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.SlaveReregisteredMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3208, - serialized_end=3268, -) - - -_UNREGISTERSLAVEMESSAGE = descriptor.Descriptor( - name='UnregisterSlaveMessage', - full_name='mesos.internal.UnregisterSlaveMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.UnregisterSlaveMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3270, - serialized_end=3328, -) - - -_HEARTBEATMESSAGE = descriptor.Descriptor( - name='HeartbeatMessage', - full_name='mesos.internal.HeartbeatMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.HeartbeatMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3330, - serialized_end=3382, -) - - -_SHUTDOWNFRAMEWORKMESSAGE = descriptor.Descriptor( - name='ShutdownFrameworkMessage', - full_name='mesos.internal.ShutdownFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ShutdownFrameworkMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3384, - serialized_end=3452, -) - - -_SHUTDOWNEXECUTORMESSAGE = descriptor.Descriptor( - name='ShutdownExecutorMessage', - full_name='mesos.internal.ShutdownExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3454, - serialized_end=3479, -) - - -_UPDATEFRAMEWORKMESSAGE = descriptor.Descriptor( - name='UpdateFrameworkMessage', - full_name='mesos.internal.UpdateFrameworkMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.UpdateFrameworkMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pid', full_name='mesos.internal.UpdateFrameworkMessage.pid', index=1, - number=2, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3481, - serialized_end=3560, -) - - -_REGISTEREXECUTORMESSAGE = descriptor.Descriptor( - name='RegisterExecutorMessage', - full_name='mesos.internal.RegisterExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.RegisterExecutorMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.RegisterExecutorMessage.executor_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3562, - serialized_end=3669, -) - - -_EXECUTORREGISTEREDMESSAGE = descriptor.Descriptor( - name='ExecutorRegisteredMessage', - full_name='mesos.internal.ExecutorRegisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='executor_info', full_name='mesos.internal.ExecutorRegisteredMessage.executor_info', index=0, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ExecutorRegisteredMessage.framework_id', index=1, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_info', full_name='mesos.internal.ExecutorRegisteredMessage.framework_info', index=2, - number=4, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ExecutorRegisteredMessage.slave_id', index=3, - number=5, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_info', full_name='mesos.internal.ExecutorRegisteredMessage.slave_info', index=4, - number=6, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3672, - serialized_end=3903, -) - - -_EXECUTORREREGISTEREDMESSAGE = descriptor.Descriptor( - name='ExecutorReregisteredMessage', - full_name='mesos.internal.ExecutorReregisteredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ExecutorReregisteredMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='slave_info', full_name='mesos.internal.ExecutorReregisteredMessage.slave_info', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=3905, - serialized_end=4006, -) - - -_EXITEDEXECUTORMESSAGE = descriptor.Descriptor( - name='ExitedExecutorMessage', - full_name='mesos.internal.ExitedExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ExitedExecutorMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ExitedExecutorMessage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.ExitedExecutorMessage.executor_id', index=2, - number=3, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='status', full_name='mesos.internal.ExitedExecutorMessage.status', index=3, - number=4, type=5, cpp_type=1, label=2, - has_default_value=False, default_value=0, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4009, - serialized_end=4164, -) - - -_RECONNECTEXECUTORMESSAGE = descriptor.Descriptor( - name='ReconnectExecutorMessage', - full_name='mesos.internal.ReconnectExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='slave_id', full_name='mesos.internal.ReconnectExecutorMessage.slave_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4166, - serialized_end=4226, -) - - -_REREGISTEREXECUTORMESSAGE = descriptor.Descriptor( - name='ReregisterExecutorMessage', - full_name='mesos.internal.ReregisterExecutorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='executor_id', full_name='mesos.internal.ReregisterExecutorMessage.executor_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.ReregisterExecutorMessage.framework_id', index=1, - number=2, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='tasks', full_name='mesos.internal.ReregisterExecutorMessage.tasks', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='updates', full_name='mesos.internal.ReregisterExecutorMessage.updates', index=3, - number=4, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4229, - serialized_end=4417, -) - - -_REGISTERPROJDMESSAGE = descriptor.Descriptor( - name='RegisterProjdMessage', - full_name='mesos.internal.RegisterProjdMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='project', full_name='mesos.internal.RegisterProjdMessage.project', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4419, - serialized_end=4458, -) - - -_PROJDREADYMESSAGE = descriptor.Descriptor( - name='ProjdReadyMessage', - full_name='mesos.internal.ProjdReadyMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='project', full_name='mesos.internal.ProjdReadyMessage.project', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4460, - serialized_end=4496, -) - - -_PROJDUPDATERESOURCESMESSAGE = descriptor.Descriptor( - name='ProjdUpdateResourcesMessage', - full_name='mesos.internal.ProjdUpdateResourcesMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='parameters', full_name='mesos.internal.ProjdUpdateResourcesMessage.parameters', index=0, - number=1, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4498, - serialized_end=4566, -) - - -_FRAMEWORKEXPIREDMESSAGE = descriptor.Descriptor( - name='FrameworkExpiredMessage', - full_name='mesos.internal.FrameworkExpiredMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_id', full_name='mesos.internal.FrameworkExpiredMessage.framework_id', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4568, - serialized_end=4635, -) - - -_SHUTDOWNMESSAGE = descriptor.Descriptor( - name='ShutdownMessage', - full_name='mesos.internal.ShutdownMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='message', full_name='mesos.internal.ShutdownMessage.message', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4637, - serialized_end=4671, -) - - -_AUTHENTICATEMESSAGE = descriptor.Descriptor( - name='AuthenticateMessage', - full_name='mesos.internal.AuthenticateMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='pid', full_name='mesos.internal.AuthenticateMessage.pid', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4673, - serialized_end=4707, -) - - -_AUTHENTICATIONMECHANISMSMESSAGE = descriptor.Descriptor( - name='AuthenticationMechanismsMessage', - full_name='mesos.internal.AuthenticationMechanismsMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='mechanisms', full_name='mesos.internal.AuthenticationMechanismsMessage.mechanisms', index=0, - number=1, type=9, cpp_type=9, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4709, - serialized_end=4762, -) - - -_AUTHENTICATIONSTARTMESSAGE = descriptor.Descriptor( - name='AuthenticationStartMessage', - full_name='mesos.internal.AuthenticationStartMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='mechanism', full_name='mesos.internal.AuthenticationStartMessage.mechanism', index=0, - number=1, type=9, cpp_type=9, label=2, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='data', full_name='mesos.internal.AuthenticationStartMessage.data', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4764, - serialized_end=4825, -) - - -_AUTHENTICATIONSTEPMESSAGE = descriptor.Descriptor( - name='AuthenticationStepMessage', - full_name='mesos.internal.AuthenticationStepMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='data', full_name='mesos.internal.AuthenticationStepMessage.data', index=0, - number=1, type=12, cpp_type=9, label=2, - has_default_value=False, default_value="", - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4827, - serialized_end=4868, -) - - -_AUTHENTICATIONCOMPLETEDMESSAGE = descriptor.Descriptor( - name='AuthenticationCompletedMessage', - full_name='mesos.internal.AuthenticationCompletedMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4870, - serialized_end=4902, -) - - -_AUTHENTICATIONFAILEDMESSAGE = descriptor.Descriptor( - name='AuthenticationFailedMessage', - full_name='mesos.internal.AuthenticationFailedMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4904, - serialized_end=4933, -) - - -_AUTHENTICATIONERRORMESSAGE = descriptor.Descriptor( - name='AuthenticationErrorMessage', - full_name='mesos.internal.AuthenticationErrorMessage', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='error', full_name='mesos.internal.AuthenticationErrorMessage.error', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4935, - serialized_end=4978, -) - - -_ARCHIVE_FRAMEWORK = descriptor.Descriptor( - name='Framework', - full_name='mesos.internal.Archive.Framework', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='framework_info', full_name='mesos.internal.Archive.Framework.framework_info', index=0, - number=1, type=11, cpp_type=10, label=2, - has_default_value=False, default_value=None, - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='pid', full_name='mesos.internal.Archive.Framework.pid', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=unicode("", "utf-8"), - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - descriptor.FieldDescriptor( - name='tasks', full_name='mesos.internal.Archive.Framework.tasks', index=2, - number=3, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=5047, - serialized_end=5154, -) - -_ARCHIVE = descriptor.Descriptor( - name='Archive', - full_name='mesos.internal.Archive', - filename=None, - file=DESCRIPTOR, - containing_type=None, - fields=[ - descriptor.FieldDescriptor( - name='frameworks', full_name='mesos.internal.Archive.frameworks', index=0, - number=1, type=11, cpp_type=10, label=3, - has_default_value=False, default_value=[], - message_type=None, enum_type=None, containing_type=None, - is_extension=False, extension_scope=None, - options=None), - ], - extensions=[ - ], - nested_types=[_ARCHIVE_FRAMEWORK, ], - enum_types=[ - ], - options=None, - is_extendable=False, - extension_ranges=[], - serialized_start=4981, - serialized_end=5154, -) - -_TASK.fields_by_name['task_id'].message_type = mesos_pb2._TASKID -_TASK.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_TASK.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_TASK.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_TASK.fields_by_name['state'].enum_type = mesos_pb2._TASKSTATE -_TASK.fields_by_name['resources'].message_type = mesos_pb2._RESOURCE -_TASK.fields_by_name['statuses'].message_type = mesos_pb2._TASKSTATUS -_STATUSUPDATE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_STATUSUPDATE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_STATUSUPDATE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_STATUSUPDATE.fields_by_name['status'].message_type = mesos_pb2._TASKSTATUS -_STATUSUPDATERECORD.fields_by_name['type'].enum_type = _STATUSUPDATERECORD_TYPE -_STATUSUPDATERECORD.fields_by_name['update'].message_type = _STATUSUPDATE -_STATUSUPDATERECORD_TYPE.containing_type = _STATUSUPDATERECORD; -_EXECUTORTOFRAMEWORKMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_EXECUTORTOFRAMEWORKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_EXECUTORTOFRAMEWORKMESSAGE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_FRAMEWORKTOEXECUTORMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_FRAMEWORKTOEXECUTORMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_FRAMEWORKTOEXECUTORMESSAGE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_REGISTERFRAMEWORKMESSAGE.fields_by_name['framework'].message_type = mesos_pb2._FRAMEWORKINFO -_REREGISTERFRAMEWORKMESSAGE.fields_by_name['framework'].message_type = mesos_pb2._FRAMEWORKINFO -_FRAMEWORKREGISTEREDMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_FRAMEWORKREGISTEREDMESSAGE.fields_by_name['master_info'].message_type = mesos_pb2._MASTERINFO -_FRAMEWORKREREGISTEREDMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_FRAMEWORKREREGISTEREDMESSAGE.fields_by_name['master_info'].message_type = mesos_pb2._MASTERINFO -_UNREGISTERFRAMEWORKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_DEACTIVATEFRAMEWORKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_RESOURCEREQUESTMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_RESOURCEREQUESTMESSAGE.fields_by_name['requests'].message_type = mesos_pb2._REQUEST -_RESOURCEOFFERSMESSAGE.fields_by_name['offers'].message_type = mesos_pb2._OFFER -_LAUNCHTASKSMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_LAUNCHTASKSMESSAGE.fields_by_name['offer_id'].message_type = mesos_pb2._OFFERID -_LAUNCHTASKSMESSAGE.fields_by_name['tasks'].message_type = mesos_pb2._TASKINFO -_LAUNCHTASKSMESSAGE.fields_by_name['filters'].message_type = mesos_pb2._FILTERS -_LAUNCHTASKSMESSAGE.fields_by_name['offer_ids'].message_type = mesos_pb2._OFFERID -_RESCINDRESOURCEOFFERMESSAGE.fields_by_name['offer_id'].message_type = mesos_pb2._OFFERID -_REVIVEOFFERSMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_RUNTASKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_RUNTASKMESSAGE.fields_by_name['framework'].message_type = mesos_pb2._FRAMEWORKINFO -_RUNTASKMESSAGE.fields_by_name['task'].message_type = mesos_pb2._TASKINFO -_KILLTASKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_KILLTASKMESSAGE.fields_by_name['task_id'].message_type = mesos_pb2._TASKID -_STATUSUPDATEMESSAGE.fields_by_name['update'].message_type = _STATUSUPDATE -_STATUSUPDATEACKNOWLEDGEMENTMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_STATUSUPDATEACKNOWLEDGEMENTMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_STATUSUPDATEACKNOWLEDGEMENTMESSAGE.fields_by_name['task_id'].message_type = mesos_pb2._TASKID -_LOSTSLAVEMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_RECONCILETASKSMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_RECONCILETASKSMESSAGE.fields_by_name['statuses'].message_type = mesos_pb2._TASKSTATUS -_REGISTERSLAVEMESSAGE.fields_by_name['slave'].message_type = mesos_pb2._SLAVEINFO -_REREGISTERSLAVEMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_REREGISTERSLAVEMESSAGE.fields_by_name['slave'].message_type = mesos_pb2._SLAVEINFO -_REREGISTERSLAVEMESSAGE.fields_by_name['executor_infos'].message_type = mesos_pb2._EXECUTORINFO -_REREGISTERSLAVEMESSAGE.fields_by_name['tasks'].message_type = _TASK -_REREGISTERSLAVEMESSAGE.fields_by_name['completed_frameworks'].message_type = _ARCHIVE_FRAMEWORK -_SLAVEREGISTEREDMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_SLAVEREREGISTEREDMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_UNREGISTERSLAVEMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_HEARTBEATMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_SHUTDOWNFRAMEWORKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_UPDATEFRAMEWORKMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_REGISTEREXECUTORMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_REGISTEREXECUTORMESSAGE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_EXECUTORREGISTEREDMESSAGE.fields_by_name['executor_info'].message_type = mesos_pb2._EXECUTORINFO -_EXECUTORREGISTEREDMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_EXECUTORREGISTEREDMESSAGE.fields_by_name['framework_info'].message_type = mesos_pb2._FRAMEWORKINFO -_EXECUTORREGISTEREDMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_EXECUTORREGISTEREDMESSAGE.fields_by_name['slave_info'].message_type = mesos_pb2._SLAVEINFO -_EXECUTORREREGISTEREDMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_EXECUTORREREGISTEREDMESSAGE.fields_by_name['slave_info'].message_type = mesos_pb2._SLAVEINFO -_EXITEDEXECUTORMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_EXITEDEXECUTORMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_EXITEDEXECUTORMESSAGE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_RECONNECTEXECUTORMESSAGE.fields_by_name['slave_id'].message_type = mesos_pb2._SLAVEID -_REREGISTEREXECUTORMESSAGE.fields_by_name['executor_id'].message_type = mesos_pb2._EXECUTORID -_REREGISTEREXECUTORMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_REREGISTEREXECUTORMESSAGE.fields_by_name['tasks'].message_type = mesos_pb2._TASKINFO -_REREGISTEREXECUTORMESSAGE.fields_by_name['updates'].message_type = _STATUSUPDATE -_PROJDUPDATERESOURCESMESSAGE.fields_by_name['parameters'].message_type = mesos_pb2._PARAMETERS -_FRAMEWORKEXPIREDMESSAGE.fields_by_name['framework_id'].message_type = mesos_pb2._FRAMEWORKID -_ARCHIVE_FRAMEWORK.fields_by_name['framework_info'].message_type = mesos_pb2._FRAMEWORKINFO -_ARCHIVE_FRAMEWORK.fields_by_name['tasks'].message_type = _TASK -_ARCHIVE_FRAMEWORK.containing_type = _ARCHIVE; -_ARCHIVE.fields_by_name['frameworks'].message_type = _ARCHIVE_FRAMEWORK -DESCRIPTOR.message_types_by_name['Task'] = _TASK -DESCRIPTOR.message_types_by_name['RoleInfo'] = _ROLEINFO -DESCRIPTOR.message_types_by_name['StatusUpdate'] = _STATUSUPDATE -DESCRIPTOR.message_types_by_name['StatusUpdateRecord'] = _STATUSUPDATERECORD -DESCRIPTOR.message_types_by_name['SubmitSchedulerRequest'] = _SUBMITSCHEDULERREQUEST -DESCRIPTOR.message_types_by_name['SubmitSchedulerResponse'] = _SUBMITSCHEDULERRESPONSE -DESCRIPTOR.message_types_by_name['ExecutorToFrameworkMessage'] = _EXECUTORTOFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['FrameworkToExecutorMessage'] = _FRAMEWORKTOEXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['RegisterFrameworkMessage'] = _REGISTERFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['ReregisterFrameworkMessage'] = _REREGISTERFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['FrameworkRegisteredMessage'] = _FRAMEWORKREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['FrameworkReregisteredMessage'] = _FRAMEWORKREREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['UnregisterFrameworkMessage'] = _UNREGISTERFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['DeactivateFrameworkMessage'] = _DEACTIVATEFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['ResourceRequestMessage'] = _RESOURCEREQUESTMESSAGE -DESCRIPTOR.message_types_by_name['ResourceOffersMessage'] = _RESOURCEOFFERSMESSAGE -DESCRIPTOR.message_types_by_name['LaunchTasksMessage'] = _LAUNCHTASKSMESSAGE -DESCRIPTOR.message_types_by_name['RescindResourceOfferMessage'] = _RESCINDRESOURCEOFFERMESSAGE -DESCRIPTOR.message_types_by_name['ReviveOffersMessage'] = _REVIVEOFFERSMESSAGE -DESCRIPTOR.message_types_by_name['RunTaskMessage'] = _RUNTASKMESSAGE -DESCRIPTOR.message_types_by_name['KillTaskMessage'] = _KILLTASKMESSAGE -DESCRIPTOR.message_types_by_name['StatusUpdateMessage'] = _STATUSUPDATEMESSAGE -DESCRIPTOR.message_types_by_name['StatusUpdateAcknowledgementMessage'] = _STATUSUPDATEACKNOWLEDGEMENTMESSAGE -DESCRIPTOR.message_types_by_name['LostSlaveMessage'] = _LOSTSLAVEMESSAGE -DESCRIPTOR.message_types_by_name['ReconcileTasksMessage'] = _RECONCILETASKSMESSAGE -DESCRIPTOR.message_types_by_name['FrameworkErrorMessage'] = _FRAMEWORKERRORMESSAGE -DESCRIPTOR.message_types_by_name['RegisterSlaveMessage'] = _REGISTERSLAVEMESSAGE -DESCRIPTOR.message_types_by_name['ReregisterSlaveMessage'] = _REREGISTERSLAVEMESSAGE -DESCRIPTOR.message_types_by_name['SlaveRegisteredMessage'] = _SLAVEREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['SlaveReregisteredMessage'] = _SLAVEREREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['UnregisterSlaveMessage'] = _UNREGISTERSLAVEMESSAGE -DESCRIPTOR.message_types_by_name['HeartbeatMessage'] = _HEARTBEATMESSAGE -DESCRIPTOR.message_types_by_name['ShutdownFrameworkMessage'] = _SHUTDOWNFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['ShutdownExecutorMessage'] = _SHUTDOWNEXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['UpdateFrameworkMessage'] = _UPDATEFRAMEWORKMESSAGE -DESCRIPTOR.message_types_by_name['RegisterExecutorMessage'] = _REGISTEREXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['ExecutorRegisteredMessage'] = _EXECUTORREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['ExecutorReregisteredMessage'] = _EXECUTORREREGISTEREDMESSAGE -DESCRIPTOR.message_types_by_name['ExitedExecutorMessage'] = _EXITEDEXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['ReconnectExecutorMessage'] = _RECONNECTEXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['ReregisterExecutorMessage'] = _REREGISTEREXECUTORMESSAGE -DESCRIPTOR.message_types_by_name['RegisterProjdMessage'] = _REGISTERPROJDMESSAGE -DESCRIPTOR.message_types_by_name['ProjdReadyMessage'] = _PROJDREADYMESSAGE -DESCRIPTOR.message_types_by_name['ProjdUpdateResourcesMessage'] = _PROJDUPDATERESOURCESMESSAGE -DESCRIPTOR.message_types_by_name['FrameworkExpiredMessage'] = _FRAMEWORKEXPIREDMESSAGE -DESCRIPTOR.message_types_by_name['ShutdownMessage'] = _SHUTDOWNMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticateMessage'] = _AUTHENTICATEMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationMechanismsMessage'] = _AUTHENTICATIONMECHANISMSMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationStartMessage'] = _AUTHENTICATIONSTARTMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationStepMessage'] = _AUTHENTICATIONSTEPMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationCompletedMessage'] = _AUTHENTICATIONCOMPLETEDMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationFailedMessage'] = _AUTHENTICATIONFAILEDMESSAGE -DESCRIPTOR.message_types_by_name['AuthenticationErrorMessage'] = _AUTHENTICATIONERRORMESSAGE -DESCRIPTOR.message_types_by_name['Archive'] = _ARCHIVE - -class Task(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _TASK - - # @@protoc_insertion_point(class_scope:mesos.internal.Task) - -class RoleInfo(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ROLEINFO - - # @@protoc_insertion_point(class_scope:mesos.internal.RoleInfo) - -class StatusUpdate(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _STATUSUPDATE - - # @@protoc_insertion_point(class_scope:mesos.internal.StatusUpdate) - -class StatusUpdateRecord(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _STATUSUPDATERECORD - - # @@protoc_insertion_point(class_scope:mesos.internal.StatusUpdateRecord) - -class SubmitSchedulerRequest(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SUBMITSCHEDULERREQUEST - - # @@protoc_insertion_point(class_scope:mesos.internal.SubmitSchedulerRequest) - -class SubmitSchedulerResponse(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SUBMITSCHEDULERRESPONSE - - # @@protoc_insertion_point(class_scope:mesos.internal.SubmitSchedulerResponse) - -class ExecutorToFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXECUTORTOFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ExecutorToFrameworkMessage) - -class FrameworkToExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKTOEXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.FrameworkToExecutorMessage) - -class RegisterFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REGISTERFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RegisterFrameworkMessage) - -class ReregisterFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REREGISTERFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReregisterFrameworkMessage) - -class FrameworkRegisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.FrameworkRegisteredMessage) - -class FrameworkReregisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKREREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.FrameworkReregisteredMessage) - -class UnregisterFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _UNREGISTERFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.UnregisterFrameworkMessage) - -class DeactivateFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _DEACTIVATEFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.DeactivateFrameworkMessage) - -class ResourceRequestMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESOURCEREQUESTMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ResourceRequestMessage) - -class ResourceOffersMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESOURCEOFFERSMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ResourceOffersMessage) - -class LaunchTasksMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _LAUNCHTASKSMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.LaunchTasksMessage) - -class RescindResourceOfferMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RESCINDRESOURCEOFFERMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RescindResourceOfferMessage) - -class ReviveOffersMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REVIVEOFFERSMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReviveOffersMessage) - -class RunTaskMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RUNTASKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RunTaskMessage) - -class KillTaskMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _KILLTASKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.KillTaskMessage) - -class StatusUpdateMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _STATUSUPDATEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.StatusUpdateMessage) - -class StatusUpdateAcknowledgementMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _STATUSUPDATEACKNOWLEDGEMENTMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.StatusUpdateAcknowledgementMessage) - -class LostSlaveMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _LOSTSLAVEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.LostSlaveMessage) - -class ReconcileTasksMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RECONCILETASKSMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReconcileTasksMessage) - -class FrameworkErrorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKERRORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.FrameworkErrorMessage) - -class RegisterSlaveMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REGISTERSLAVEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RegisterSlaveMessage) - -class ReregisterSlaveMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REREGISTERSLAVEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReregisterSlaveMessage) - -class SlaveRegisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SLAVEREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.SlaveRegisteredMessage) - -class SlaveReregisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SLAVEREREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.SlaveReregisteredMessage) - -class UnregisterSlaveMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _UNREGISTERSLAVEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.UnregisterSlaveMessage) - -class HeartbeatMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _HEARTBEATMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.HeartbeatMessage) - -class ShutdownFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SHUTDOWNFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ShutdownFrameworkMessage) - -class ShutdownExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SHUTDOWNEXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ShutdownExecutorMessage) - -class UpdateFrameworkMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _UPDATEFRAMEWORKMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.UpdateFrameworkMessage) - -class RegisterExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REGISTEREXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RegisterExecutorMessage) - -class ExecutorRegisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXECUTORREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ExecutorRegisteredMessage) - -class ExecutorReregisteredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXECUTORREREGISTEREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ExecutorReregisteredMessage) - -class ExitedExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _EXITEDEXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ExitedExecutorMessage) - -class ReconnectExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _RECONNECTEXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReconnectExecutorMessage) - -class ReregisterExecutorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REREGISTEREXECUTORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ReregisterExecutorMessage) - -class RegisterProjdMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _REGISTERPROJDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.RegisterProjdMessage) - -class ProjdReadyMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _PROJDREADYMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ProjdReadyMessage) - -class ProjdUpdateResourcesMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _PROJDUPDATERESOURCESMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ProjdUpdateResourcesMessage) - -class FrameworkExpiredMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _FRAMEWORKEXPIREDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.FrameworkExpiredMessage) - -class ShutdownMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _SHUTDOWNMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.ShutdownMessage) - -class AuthenticateMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATEMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticateMessage) - -class AuthenticationMechanismsMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONMECHANISMSMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationMechanismsMessage) - -class AuthenticationStartMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONSTARTMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationStartMessage) - -class AuthenticationStepMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONSTEPMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationStepMessage) - -class AuthenticationCompletedMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONCOMPLETEDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationCompletedMessage) - -class AuthenticationFailedMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONFAILEDMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationFailedMessage) - -class AuthenticationErrorMessage(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _AUTHENTICATIONERRORMESSAGE - - # @@protoc_insertion_point(class_scope:mesos.internal.AuthenticationErrorMessage) - -class Archive(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - - class Framework(message.Message): - __metaclass__ = reflection.GeneratedProtocolMessageType - DESCRIPTOR = _ARCHIVE_FRAMEWORK - - # @@protoc_insertion_point(class_scope:mesos.internal.Archive.Framework) - DESCRIPTOR = _ARCHIVE - - # @@protoc_insertion_point(class_scope:mesos.internal.Archive) - -# @@protoc_insertion_point(module_scope) diff --git a/dpark/pymesos/process.py b/dpark/pymesos/process.py deleted file mode 100644 index bfdc6235..00000000 --- a/dpark/pymesos/process.py +++ /dev/null @@ -1,268 +0,0 @@ -import sys, os -import socket -import time -import threading -import logging -import Queue -import select - -from mesos_pb2 import * -from messages_pb2 import * - -logger = logging.getLogger(__name__) - -def spawn(target, *args, **kw): - t = threading.Thread(target=target, name=target.__name__, args=args, kwargs=kw) - t.daemon = True - t.start() - return t - -class UPID(object): - def __init__(self, name, addr=None): - if addr is None and name and '@' in name: - name, addr = name.split('@') - self.name = name - self.addr = addr - - def __str__(self): - return "%s@%s" % (self.name, self.addr) - - -def async(f): - def func(self, *a, **kw): - self.delay(0, f, self, *a, **kw) - return func - -class Process(UPID): - def __init__(self, name, port=0): - UPID.__init__(self, name) - self.port = port - self.conn_pool = {} - self.jobs = Queue.PriorityQueue() - self.linked = {} - self.sender = None - self.aborted = False - - def delay(self, delay, func, *args, **kw): - self.jobs.put((time.time() + delay, 0, func, args, kw)) - - def run_jobs(self): - while True: - try: - job = self.jobs.get(timeout=1) - except Queue.Empty: - if self.aborted: - break - continue - - #self.jobs.task_done() - t, tried, func, args, kw = job - now = time.time() - if t > now: - if self.aborted: - break - self.jobs.put(job) - time.sleep(min(t-now, 0.1)) - continue - - try: - #logger.debug("run job %s", func.__name__) - func(*args, **kw) - #logger.debug("run job %s comeplete", func.__name__) - except Exception, e: - logger.error("error while call %s (tried %d times)", func, tried) - import traceback; traceback.print_exc() - if tried < 4: - self.jobs.put((t + 3 ** tried, tried + 1, func, args, kw)) - - @async - def link(self, upid, callback): - self._get_conn(upid.addr) - self.linked[upid.addr] = callback - - def _get_conn(self, addr): - if addr not in self.conn_pool: - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - host, port = addr.split(':') - s.connect((host, int(port))) - self.conn_pool[addr] = s - return self.conn_pool[addr] - - def _encode(self, upid, msg): - if isinstance(msg, str): - body = '' - uri = '/%s/%s' % (upid.name, msg) - else: - body = msg.SerializeToString() - uri = '/%s/mesos.internal.%s' % (upid.name, msg.__class__.__name__) - agent = 'libprocess/%s@%s' % (self.name, self.addr) - msg = ['POST %s HTTP/1.0' % str(uri), - 'User-Agent: %s' % agent, - 'Connection: Keep-Alive',] - if body: - msg += [ - 'Transfer-Encoding: chunked', - '', - '%x' % len(body), body, - '0'] - msg += ['', ''] # for last \r\n\r\n - return '\r\n'.join(msg) - - #@async - def send(self, upid, msg): - logger.debug("send to %s %s", upid, msg.__class__.__name__) - data = self._encode(upid, msg) - try: - conn = self._get_conn(upid.addr) - conn.send(data) - except IOError: - logger.warning("failed to send data to %s, retry again", upid) - self.conn_pool.pop(upid.addr, None) - if upid.addr in self.linked: # broken link - callback = self.linked.pop(upid.addr) - callback() - raise - - def reply(self, msg): - return self.send(self.sender, msg) - - def onPing(self): - self.reply('PONG') - - @async - def handle(self, msg): - if self.aborted: - return - name = msg.__class__.__name__ - f = getattr(self, 'on' + name, None) - assert f, 'should have on%s()' % name - args = [v for (_,v) in msg.ListFields()] - f(*args) - - def abort(self): - self.aborted = True - self.listen_sock.close() - - def stop(self): - self.abort() - self.join() - for addr in self.conn_pool: - self.conn_pool[addr].close() - self.conn_pool.clear() - - def join(self): - self.delay_t.join() - return self.accept_t.join() - - def run(self): - self.start() - return self.join() - - def start(self): - self.listen_sock = sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - sock.setsockopt(socket.SOL_SOCKET, socket.SO_REUSEADDR, 1) - sock.bind(('0.0.0.0', self.port)) - if not self.port: - port = sock.getsockname()[1] - self.addr = '%s:%d' % (socket.gethostname(), port) - self.accept_t = spawn(self.ioloop, sock) - self.delay_t = spawn(self.run_jobs) - - def process_message(self, rf): - headers = [] - while True: - try: - line = rf.readline() - except IOError: - break - if not line or line == '\r\n': - break - headers.append(line) - if not headers: - return False # EoF - - method, uri, _ = headers[0].split(' ') - _, process, mname = uri.split('/') - assert process == self.name, 'unexpected messages' - agent = headers[1].split(' ')[1].strip() - logger.debug("incoming request: %s from %s", uri, agent) - - sender_name, addr = agent.split('@') - self.sender = UPID(sender_name.split('/')[1], addr) - - if mname == 'PING': - self.onPing() - return True - - size = rf.readline() - if size: - size = int(size, 16) - body = rf.read(size+2)[:-2] - rf.read(5) # ending - else: - body = '' - - sname = mname.split('.')[2] - if sname not in globals(): - logger.error("unknown messages: %s", sname) - return True - - try: - msg = globals()[sname].FromString(body) - self.handle(msg) - except Exception, e: - logger.error("error while processing message %s: %s", sname, e) - import traceback; traceback.print_exc() - return True - - def ioloop(self, sock): - sock.listen(64) - sfd = sock.fileno() - conns = {sfd: sock} - while not self.aborted: - rlist = select.select(conns.keys(), [], [], 1)[0] - for fd in rlist: - if fd == sfd: - conn, addr = sock.accept() - logger.debug("accepted conn from %s", addr) - conns[conn.fileno()] = conn - elif fd in conns: - try: - f = conns[fd].makefile('r') - while True: - if not self.process_message(f): - conns.pop(fd).close() - break - # is there any data in read buffer ? - if not f._rbuf.tell(): - break - f.close() - except Exception, e: - import traceback; traceback.print_exc() - conns.pop(fd).close() - - sock.close() - """ - def communicate(self, conn): - rf = conn.makefile('r', 4096) - while not self.aborted: - cont = self.process_message(rf) - if not cont: - break - rf.close() - - def ioloop(self, s): - s.listen(1) - conns = [] - while True: - conn, addr = s.accept() - logger.debug("accepted conn from %s", addr) - conns.append(conn) - if self.aborted: - break - - spawn(self.communicate, conn) - - for c in conns: - c.close() - """ diff --git a/dpark/pymesos/scheduler.py b/dpark/pymesos/scheduler.py deleted file mode 100644 index 69629e6f..00000000 --- a/dpark/pymesos/scheduler.py +++ /dev/null @@ -1,247 +0,0 @@ -import os, sys -import time -import getpass -import logging -import struct -import socket - -from process import UPID, Process, async - -from mesos_pb2 import TASK_LOST, MasterInfo -from messages_pb2 import (RegisterFrameworkMessage, ReregisterFrameworkMessage, - DeactivateFrameworkMessage, UnregisterFrameworkMessage, - ResourceRequestMessage, ReviveOffersMessage, LaunchTasksMessage, KillTaskMessage, - StatusUpdate, StatusUpdateAcknowledgementMessage, FrameworkToExecutorMessage) - -logger = logging.getLogger(__name__) - -class Scheduler(object): - def registered(self, driver, framework_id, masterInfo): pass - def reregistered(self, driver, masterInfo): pass - def disconnected(self, driver): pass - def frameworkMessage(self, driver, slave_id, executor_id, message): pass - def resourceOffers(self, driver, offers): pass - def offerRescinded(self, driver, offer_id): pass - def statusUpdate(self, driver, status): pass - def executorLost(self, driver, executor_id, slave_id, status): pass - def slaveLost(self, driver, slave_id): pass - def error(self, driver, message): pass - -class SchedulerDriver(object): - def start(self): pass - def join(self): pass - def run(self): pass - def abort(self): pass - def stop(self, failover=False): pass - def reviveOffers(self): pass - def requestResources(self, requests): pass - def declineOffer(self, offerId, filters=None): pass - def launchTasks(self, offerId, tasks, filters=None): pass - def killTask(self, taskId): pass - def sendFrameworkMessage(self, executorId, slaveId, data): pass - - -class MesosSchedulerDriver(Process): - def __init__(self, sched, framework, master_uri): - Process.__init__(self, 'scheduler') - self.sched = sched - #self.executor_info = executor_info - self.master_uri = master_uri - self.framework = framework - self.framework.failover_timeout = 100 - self.framework_id = framework.id - - self.master = None - self.detector = None - - self.connected = False - self.savedOffers = {} - self.savedSlavePids = {} - - @async # called by detector - def onNewMasterDetectedMessage(self, data): - try: - info = MasterInfo() - info.ParseFromString(data) - ip = socket.inet_ntoa(struct.pack(' 15: - id = fmt % (task.task_id.value, task.command.value[:12] + '...') - else: - id = fmt % (task.task_id.value, task.command.value) - info.executor_id.value = id - info.framework_id.value = self.id - info.command.value = task.command.value - return info - - def createExecutor(self, info, directory): - executor = Executor(self.id, info, directory) - self.executors[executor.id.value] = executor - return executor - - def destroyExecutor(self, exec_id): - self.executors.pop(exec_id.value, None) - - def get_executor(self, exec_id): - if isinstance(exec_id, TaskID): - tid = exec_id.value - for executor in self.executors.values(): - if tid in executor.queuedTasks or tid in executor.launchedTasks: - return executor - return - return self.executors.get(exec_id.value) - getExecutor = get_executor - - -class ProcessInfo(object): - def __init__(self, fid, eid, pid, directory): - self.framework_id = fid - self.executor_id = eid - self.pid = pid - self.directory = directory - - -class IsolationModule(object): - def __init__(self): - self.infos = {} - self.pids = {} - - def initialize(self, slave): - self.slave = slave - t = threading.Thread(target=self.reaper) - t.daemon = True - t.start() - - def launchExecutor(self, framework_id, info, executor_info, directory, resources): - fid = framework_id.value - eid = executor_info.executor_id.value - logger.info("Launching %s (%s) in %s with resources %s for framework %s", - eid, executor_info.command.value, directory, resources, fid) - pid = os.fork() - assert pid >= 0 - if pid: - logger.info("Forked executor at %d", pid) - pinfo = ProcessInfo(framework_id, executor_info.executor_id, pid, directory) - self.infos.setdefault(fid, {})[eid] = pinfo - self.pids[pid] = pinfo - self.slave.executorStarted(framework_id, executor_info.executor_id, pid) - else: - #pid = os.setsid() - # copy UPID of slave - launcher = Launcher(framework_id, executor_info.executor_id, executor_info.command, - info.user, directory, self.slave) - launcher.run() - - def killExecutor(self, framework_id, executor_id): - pinfo = self.infos.get(framework_id.value, {}).get(executor_id.value) - if pinfo is None: - logger.error("ERROR! Asked to kill an unknown executor! %s", executor_id.value) - return - pid = pinfo.pid - os.kill(pid, signal.SIGKILL) - # remove when waitpid() - #self._removeExecutor(pinfo) - - def removeExecutor(self, e): - logger.info("remove executor: %s", e) - if e.framework_id.value not in self.infos: - return - self.infos[e.framework_id.value].pop(e.executor_id.value, None) - if not self.infos[e.framework_id.value]: - del self.infos[e.framework_id.value] - self.pids.pop(e.pid) - - def resourcesChanged(self, framework_id, executor_id, resources): - pass - - def reaper(self): - while True: - if not self.pids: - time.sleep(1) - continue - - try: - pid, status = os.waitpid(-1, 0) - except OSError, e: - time.sleep(1) - logger.error("waitpid: %s", e) - continue - - if pid > 0 and pid in self.pids and not os.WIFSTOPPED(status): - e = self.pids[pid] - logger.info("Telling slave of lost executor %s of framework %s", - e.executor_id, e.framework_id) - self.slave.executorExited(e.framework_id, e.executor_id, status) - self.removeExecutor(e) - - -def isTerminalTaskState(state): - return state in (TASK_FINISHED, TASK_FAILED, TASK_KILLED, TASK_LOST) - - -class Slave(Process): - def __init__(self, options): - Process.__init__(self, "slave") - self.options = options - self.resources = Resources(options.cpus, options.mem) - self.attributes = options.attributes - - self.id = None - self.isolation = IsolationModule() - self.info = self.getSlaveInfo() - self.master = UPID('master', options.master) - self.frameworks = {} - self.startTime = time.time() - self.connected = False - - def getSlaveInfo(self): - info = SlaveInfo() - info.hostname = socket.gethostname() - info.webui_hostname = '' - info.webui_port = 8081 - cpus = info.resources.add() - cpus.name = 'cpus' - cpus.type = 0 - cpus.scalar.value = self.resources.cpus - mem = info.resources.add() - mem.name = 'mem' - mem.type = 0 - mem.scalar.value = self.resources.mem - if self.attributes: - for attrs in self.attributes.split(','): - name,value = attrs.split(':') - a = self.info.attributes.add() - a.name = name - a.type = Value.TEXT # 3 # TEXT - a.text.value = value - return info - - def getFramework(self, framework_id): - return self.frameworks.get(framework_id.value) - - def onNewMasterDetectedMessage(self, pid): - self.master = UPID(pid) - self.register() - - def onNoMasterDetectedMessage(self): - self.master = None - self.connected = False - - def register(self): - if self.id is None: - msg = RegisterSlaveMessage() - msg.slave.MergeFrom(self.info) - else: - msg = ReregisterSlaveMessage() - msg.slave_id.MergeFrom(self.id) - msg.slave.MergeFrom(self.info) - for framework in self.frameworks.itervalues(): - for executor in framework.executors.values(): - msg.executor_infos.add().MergeFrom(executor.info) - for task in executor.launchedTasks.itervalues(): - msg.tasks.add().MergeFrom(task) - return self.send(self.master, msg) - - def onSlaveRegisteredMessage(self, slave_id): - logger.info("slave registed %s", slave_id.value) - self.id = slave_id - self.connected = True - - def onSlaveReregisteredMessage(self, slave_id): - assert self.id == slave_id - self.connected = True - - def onRunTaskMessage(self, framework_id, framework_info, pid, task): - logger.info("Got assigned task %s for framework %s", - task.task_id.value, framework_id.value) - fid = framework_id.value - if fid not in self.frameworks: - framework = Framework(framework_id, framework_info, UPID(pid), self.options) - self.frameworks[fid] = framework - else: - framework = self.frameworks[fid] - - executorInfo = framework.get_executor_info(task) - eid = executorInfo.executor_id - executor = framework.getExecutor(eid) - if executor: - if executor.shutdown: - logger.warning("WARNING! executor is shuting down") - elif not executor.pid: - executor.queuedTasks[task.task_id.value] = task - else: - executor.addTask(task) - #self.isolation.resourcesChanged(framework_id, executor.id, executor.resources) - - msg = RunTaskMessage() - msg.framework.MergeFrom(framework.info) - msg.framework_id.MergeFrom(framework_id) - msg.pid = str(framework.pid) - msg.task.MergeFrom(task) - self.send(executor.pid, msg) - else: - directory = self.createUniqueWorkDirectory(framework.id, eid) - executor = framework.createExecutor(executorInfo, directory) - executor.queuedTasks[task.task_id.value] = task - self.isolation.launchExecutor(framework.id, framework.info, executor.info, - directory, executor.resources) - - def onKillTaskMessage(self, framework_id, task_id): - framework = self.getFramework(framework_id) - if not framework or not framework.getExecutor(task_id): - msg = StatusUpdateMessage() - update = msg.update - update.framework_id.MergeFrom(framework_id) - update.slave_id.MergeFrom(self.id) - update.status.task_id.MergeFrom(task_id) - update.status.state = TASK_LOST - update.timestamp = time.time() - update.uuid = os.urandom(16) - return self.send(self.master, msg) - - executor = framework.getExecutor(task_id) - if not executor.pid: - executor.removeTask(task_id) - msg = StatusUpdateMessage() - update = msg.update - update.framework_id.MergeFrom(framework_id) - update.slave_id.MergeFrom(self.id) - update.status.task_id.MergeFrom(task_id) - update.status.state = TASK_KILLED - update.timestamp = time.time() - update.uuid = os.urandom(16) - return self.send(self.master, msg) - - msg = KillTaskMessage() - msg.framework_id.MergeFrom(framework_id) - msg.task_id.MergeFrom(task_id) - return self.send(executor.pid, msg) - - def onShutdownFrameworkMessage(self, framework_id): - framework = self.getFramework(framework_id) - if framework: - for executor in framework.executors.values(): - self.shutdownExecutor(framework, executor) - - def shutdownExecutor(self, framework, executor): - logger.info("shutdown %s %s", framework.id.value, executor.id.value) - self.send(executor.pid, ShutdownExecutorMessage()) - executor.shutdown = True - - # delay check TODO - time.sleep(3) - self.isolation.killExecutor(framework.id, executor.id) - framework.destroyExecutor(executor.id) - if not framework.executors and not framework.updates: - self.frameworks.pop(framework.id.value) - - def onUpdateFrameworkMessage(self, framework_id, pid): - framework = self.getFramework(framework_id) - if framework: - framework.pid = pid - - def executorStarted(self, framework_id, executor_id, pid): - pass - - def executorExited(self, framework_id, executor_id, status): - logger.info("Executor %s of framework %s exited with %d", - executor_id.value, framework_id.value, status) - framework = self.getFramework(framework_id) - if not framework: - return - executor = framework.getExecutor(executor_id) - if not executor: - return - - isCommandExecutor = False - for task in executor.launchedTasks.values(): - if not isTerminalTaskState(task.state): - isCommandExecutor = not task.HasField('executor_id') - self.transitionLiveTask(task.task_id, executor_id, - framework_id, isCommandExecutor, status) - - for task in executor.queuedTasks.values(): - isCommandExecutor = task.HasField('command') - self.transitionLiveTask(task.task_id, executor_id, - framework_id, isCommandExecutor, status) - - if not isCommandExecutor: - msg = ExitedExecutorMessage() - msg.slave_id.MergeFrom(self.id) - msg.framework_id.MergeFrom(framework_id) - msg.executor_id.MergeFrom(executor_id) - msg.status = status - self.send(self.master, msg) - framework.destroyExecutor(executor_id) - - def onRegisterExecutorMessage(self, framework_id, executor_id): - framework = self.getFramework(framework_id) - if not framework: - # TODO shutdown executor - return - executor = framework.getExecutor(executor_id) - if not executor or executor.pid or executor.shutdown: - # TODO shutdown executor - return - executor.pid = self.sender - - msg = ExecutorRegisteredMessage() - msg.executor_info.MergeFrom(executor.info) - msg.framework_id.MergeFrom(framework_id) - msg.framework_info.MergeFrom(framework.info) - msg.slave_id.MergeFrom(self.id) - msg.slave_info.MergeFrom(self.info) - self.send(executor.pid, msg) - - for task in executor.queuedTasks.values(): - msg = RunTaskMessage() - msg.framework_id.MergeFrom(framework.id) - msg.framework.MergeFrom(framework.info) - msg.pid = str(framework.pid) - msg.task.MergeFrom(task) - self.send(executor.pid, msg) - - for task in executor.queuedTasks.values(): - executor.addTask(task) - executor.queuedTasks.clear() - - def onStatusUpdateMessage(self, update): - status = update.status - framework = self.getFramework(update.framework_id) - if not framework: - return - executor = framework.getExecutor(status.task_id) - if not executor: - return - executor.updateTaskState(status.task_id, status.state) - if isTerminalTaskState(status.state): - executor.removeTask(status.task_id) - - msg = StatusUpdateMessage() - msg.update.MergeFrom(update) - msg.pid = str(self) # pid - self.send(self.master, msg) - - framework.updates[update.uuid] = update - - def onStatusUpdateAcknowledgementMessage(self, slave_id, framework_id, task_id, uuid): - framework = self.getFramework(framework_id) - if framework and uuid in framework.updates: - framework.updates.pop(uuid) - if not framework.executors and not framework.updates: - self.frameworks.pop(framework_id.value) - - def onFrameworkToExecutorMessage(self, slave_id, framework_id, executor_id, data): - framework = self.getFramework(framework_id) - if not framework: - return - executor = framework.getExecutor(executor_id) - if not executor: - return - if not executor.pid: - return - msg = FrameworkToExecutorMessage() - msg.slave_id.MergeFrom(slave_id) - msg.framework_id.MergeFrom(framework_id) - msg.executor_id.MergeFrom(executor_id) - msg.data = data - self.send(executor.pid, msg) - - def onExecutorToFrameworkMessage(self, slave_id, framework_id, executor_id, data): - framework = self.getFramework(framework_id) - if not framework: - return - msg = ExecutorToFrameworkMessage() - msg.slave_id.MergeFrom(slave_id) - msg.framework_id.MergeFrom(framework_id) - msg.executor_id.MergeFrom(executor_id) - msg.data = data - self.send(framework_id.pid, msg) - - def onShutdownMessage(self): - self.stop() - - def start(self): - Process.start(self) - self.isolation.initialize(self) - self.register() # master detector TODO - - def onPing(self): - self.reply("PONG") - - def createStatusUpdate(self, task_id, executor_id, framework_id, - taskState, reason): - status = TaskStatus() - status.task_id.MergeFrom(task_id) - status.state = taskState - status.message = reason - - update = StatusUpdate() - update.framework_id.MergeFrom(framework_id) - update.slave_id.MergeFrom(self.id) - update.executor_id.MergeFrom(executor_id) - update.status.MergeFrom(status) - update.timestamp = time.time() - update.uuid = os.urandom(16) - - return update - - def statusUpdate(self, update): - logger.info("status update") - status = update.status - framework = self.getFramework(update.framework_id) - if not framework: - return - executor = framework.getExecutor(status.task_id) - if not executor: - return - if isTerminalTaskState(status.state): - executor.removeTask(status.task_id) - msg = StatusUpdateMessage() - msg.update.MergeFrom(update) - msg.pid = str(self) - self.send(self.master, msg) - # check - framework.updates[update.uuid] = update - - def transitionLiveTask(self, task_id, executor_id, framework_id, - isCommandExecutor, status): - if isCommandExecutor: - update = self.createStatusUpdate(task_id, executor_id, - framework_id, TASK_FAILED, "Executor running the task's command failed") - else: - update = self.createStatusUpdate(task_id, executor_id, - framework_id, TASK_LOST, "Executor exited") - self.statusUpdate(update) - - def createUniqueWorkDirectory(self, framework_id, executor_id): - root = self.options.work_dir - path = os.path.join(root, 'slaves', self.id.value, - 'frameworks', framework_id.value, - 'executors', executor_id.value, 'runs') - for i in range(10000): - p = os.path.join(path, str(i)) - if not os.path.exists(p): - os.makedirs(p) - return p - -def main(): - import optparse - parser = optparse.OptionParser(usage="Usage: %prog [options]") - parser.add_option("-s", "--master", type="string", default="localhost:5050", - help="May be one of:\n" - " host[:5050]" - " host:port\n" - " zk://host1:port1,host2:port2,.../path\n" - " zk://username:password@host1:port1,host2:port2,.../path\n" - " file://path/to/file (where file contains one of the above)") - parser.add_option("-c", "--cpus", type="int", default=2) - parser.add_option("-m", "--mem", type="int", default=1024) - parser.add_option("-a", "--attributes", type="string") - parser.add_option("-w", "--work_dir", type="string", default="/tmp/mesos") - parser.add_option("-q", "--quiet", action="store_true") - parser.add_option("-v", "--verbose", action="store_true") - - options, args = parser.parse_args() - logging.basicConfig(format='[slave] %(asctime)-15s %(message)s', - level=options.quiet and logging.ERROR - or options.verbose and logging.DEBUG - or logging.WARNING) - - slave = Slave(options) - slave.run() - -if __name__ == '__main__': - main() diff --git a/dpark/pymesos/zkpython.py b/dpark/pymesos/zkpython.py deleted file mode 100644 index 1ec03b5a..00000000 --- a/dpark/pymesos/zkpython.py +++ /dev/null @@ -1,115 +0,0 @@ -import zookeeper -import threading -import logging - -logger = logging.getLogger(__name__) -logger.setLevel(logging.WARNING) -zookeeper.set_debug_level(zookeeper.LOG_LEVEL_WARN) - - - -# Mapping of connection state values to human strings. -STATE_NAME_MAPPING = { - zookeeper.ASSOCIATING_STATE: "associating", - zookeeper.AUTH_FAILED_STATE: "auth-failed", - zookeeper.CONNECTED_STATE: "connected", - zookeeper.CONNECTING_STATE: "connecting", - zookeeper.EXPIRED_SESSION_STATE: "expired", -} - -# Mapping of event type to human string. -TYPE_NAME_MAPPING = { - zookeeper.NOTWATCHING_EVENT: "not-watching", - zookeeper.SESSION_EVENT: "session", - zookeeper.CREATED_EVENT: "created", - zookeeper.DELETED_EVENT: "deleted", - zookeeper.CHANGED_EVENT: "changed", - zookeeper.CHILD_EVENT: "child", -} -class TimeoutException( zookeeper.ZooKeeperException): - pass - -def logevent(h,typ, state, path): - logger.debug("event,handle:%d, type:%s, state:%s, path:%s", h, TYPE_NAME_MAPPING.get(typ, "unknown"), STATE_NAME_MAPPING.get(state, "unknown"), path) - -class ZKClient: - def __init__(self, servers, timeout= 10): - self.timeout = timeout - self.connected = False - self.handle = -1 - self.servers = servers - self.watchers = set() - self._lock = threading.Lock() - self.conn_cv = threading.Condition() - - def start(self): - self.handle = zookeeper.init(self.servers, self.connection_watcher, self.timeout * 1000) - self.conn_cv.acquire() - self.conn_cv.wait(self.timeout) - self.conn_cv.release() - if not self.connected: - raise TimeoutException - - def stop(self): - return zookeeper.close(self.handle) - - def connection_watcher(self, h, typ, state, path): - logevent(h, typ, state, path) - if typ == zookeeper.SESSION_EVENT: - if state == zookeeper.CONNECTED_STATE: - self.handle = h - with self._lock: - self.connected = True - watchers = list(self.watchers) - for watcher in watchers: - watcher.watch() - - self.conn_cv.acquire() - self.conn_cv.notifyAll() - self.conn_cv.release() - - def del_watcher(self, watcher): - with self._lock: - self.watchers.discard(watcher) - - def add_watcher(self, watcher): - with self._lock: - self.watchers.add(watcher) - if self.connected: - watcher.watch() - -class DataWatch: - def __init__(self, client, path, func): - self._client = client - self._path = path - self._func = func - self._stopped = False - client.add_watcher(self) - - def watcher(self, h, typ, state, path): - logevent(h, typ, state, path) - self.watch() - - def _do(self): - data, stat = zookeeper.get(self._client.handle, self._path, self.watcher) - return self._func(data, stat) - - def watch(self): - if self._stopped: - return - try: - result = self._do() - if result is False: - self._stopped = True - except zookeeper.NoNodeException: - raise - except zookeeper.ZooKeeperException as e: - logger.error("ZooKeeperException, type:%s, msg: %s", type(e), e) - - -class ChildrenWatch(DataWatch): - def _do(self): - children = zookeeper.get_children(self._client.handle, self._path, self.watcher) - return self._func(children) - - diff --git a/dpark/rdd.py b/dpark/rdd.py index 08685fc7..6fe3e0c0 100644 --- a/dpark/rdd.py +++ b/dpark/rdd.py @@ -1,36 +1,65 @@ +from __future__ import absolute_import +from __future__ import print_function import sys import os, os.path import time import socket import csv -from cStringIO import StringIO import itertools -import operator +import collections import math -import cPickle +import six +from six.moves import filter, map, range, zip, cPickle import random import bz2 import gzip import zlib -import logging from copy import copy import shutil import heapq import struct +import tempfile -from dpark.serialize import load_func, dump_func +try: + from cStringIO import StringIO + + BytesIO = StringIO +except ImportError: + from six import BytesIO, StringIO + +try: + from cPickle import Pickler, Unpickler +except ImportError: + from pickle import Pickler, Unpickler + +import dpark.conf from dpark.dependency import * -from dpark.util import spawn, chain -from dpark.shuffle import Merger, CoGroupMerger +from dpark.utils import ( + spawn, chain, mkdir_p, recursion_limit_breaker, atomic_file, + AbortFileReplacement, portable_hash, + masked_crc32c +) +from dpark.utils import DparkUserFatalError +from dpark.utils.log import get_logger +from dpark.utils.frame import Scope, func_info +from dpark.shuffle import SortShuffleFetcher, Merger from dpark.env import env -from dpark import moosefs +from dpark.file_manager import open_file, CHUNKSIZE +from dpark.utils.beansdb import BeansdbReader, BeansdbWriter +from contextlib import closing +from functools import reduce + +if not six.PY2: + from io import TextIOWrapper + +logger = get_logger(__name__) -logger = logging.getLogger(__name__) class Split(object): def __init__(self, idx): self.index = idx + def cached(func): def getstate(self): d = getattr(self, '_pickle_cache', None) @@ -38,49 +67,102 @@ def getstate(self): d = func(self) self._pickle_cache = d return d + return getstate + class RDD(object): def __init__(self, ctx): self.ctx = ctx - self.id = RDD.newId() + self.id = RDD.new_id() self._splits = [] - self.dependencies = [] + self._dependencies = [] self.aggregator = None self._partitioner = None self.shouldCache = False - self.snapshot_path = None + self.checkpoint_path = None + self._checkpoint_rdd = None ctx.init() - self.err = ctx.options.err self.mem = ctx.options.mem + self.cpus = 0 + self.gpus = 0 + self._preferred_locs = {} + self.repr_name = '<%s>' % (self.__class__.__name__,) + self.scope = Scope.get(self.__class__.__name__) + self.rddconf = None + self.lineage = self.scope.stackhash + self._dep_lineage_counts = None # map "dep rdd id with uniq lineages" to their counts + + self.err_ratio = ctx.options.err + self.allow_err = self.err_ratio > 1e-8 + self.exc_info = (None, None, None) nextId = 0 + @classmethod - def newId(cls): + def new_id(cls): cls.nextId += 1 return cls.nextId @cached def __getstate__(self): d = dict(self.__dict__) - d.pop('dependencies', None) + d.pop('_dependencies', None) d.pop('_splits', None) + d.pop('_preferred_locs', None) + d.pop('_dep_lineage_counts', None) d.pop('ctx', None) + d['_split_size'] = len(self.splits) return d + @property + def dep_lineage_counts(self): + if self._dep_lineage_counts is None: + lineages = collections.defaultdict(list) + for dep in self._dependencies: + rdd = dep.rdd + lineages[rdd.lineage].append(rdd) + self._dep_lineage_counts = dict([(rs[0].id, len(rs)) for rs in lineages.values()]) + return self._dep_lineage_counts + + @property + def params(self): + return None + def __len__(self): + if hasattr(self, '_split_size'): + return self._split_size return len(self.splits) - def __getslice__(self, i,j): + def __repr__(self): + return self.repr_name + + def __getslice__(self, i, j): return SliceRDD(self, i, j) + def __getitem__(self, idx): + if isinstance(idx, slice) and idx.step is None: + start = idx.start or 0 + stop = idx.stop or sys.maxsize + return self.__getslice__(start, stop) + + raise ValueError('Unsupported slicing!') + def mergeSplit(self, splitSize=None, numSplits=None): return MergedRDD(self, splitSize, numSplits) @property def splits(self): + if self._checkpoint_rdd: + return self._checkpoint_rdd.splits return self._splits + @property + def dependencies(self): + if self._checkpoint_rdd: + return self._checkpoint_rdd.dependencies + return self._dependencies + def compute(self, split): raise NotImplementedError @@ -88,48 +170,90 @@ def compute(self, split): def partitioner(self): return self._partitioner - def _preferredLocations(self, split): - return [] + + @property + def ui_label(self): + return "{}[{}]".format(self.__class__.__name__, len(self)) def cache(self): self.shouldCache = True - self._pickle_cache = None # clear pickle cache + self._pickle_cache = None # clear pickle cache return self def preferredLocations(self, split): + if self._checkpoint_rdd: + return self._checkpoint_rdd._preferred_locs.get(split, []) + if self.shouldCache: - locs = env.cacheTracker.getCachedLocs(self.id, split.index) + locs = env.cacheTrackerServer.getCachedLocs(self.id, split.index) if locs: return locs - return self._preferredLocations(split) + return self._preferred_locs.get(split, []) - def snapshot(self, path=None): + def checkpoint(self, path=None): if path is None: - path = self.ctx.options.snapshot_dir + path = self.ctx.options.checkpoint_dir if path: ident = '%d_%x' % (self.id, hash(str(self))) path = os.path.join(path, ident) - if not os.path.exists(path): - try: os.makedirs(path) - except OSError: pass - self.snapshot_path = path + mkdir_p(path) + self.checkpoint_path = path + else: + logger.warning('No checkpoint will be saved without checkpoint_dir,' + 'please re-run with --checkpoint-dir to enable checkpoint') return self + def _clear_dependencies(self): + self._dependencies = [] + self._splits = [] + + def _do_checkpoint(self): + if self.checkpoint_path: + if not self._checkpoint_rdd: + _generated = list(map(int, CheckpointRDD.generated_files(self.checkpoint_path))) + if len(_generated) != len(self): + missing = [sp.index for sp in self.splits if sp.index not in _generated] + sum(self.ctx.runJob(self, lambda x: list(x), missing), []) + + self._pickle_cache = None + self._checkpoint_rdd = CheckpointRDD(self.ctx, self.checkpoint_path) + self._clear_dependencies() + return False + return True + + @recursion_limit_breaker def iterator(self, split): - if self.snapshot_path: - p = os.path.join(self.snapshot_path, str(split.index)) - if os.path.exists(p): - v = cPickle.loads(open(p).read()) + + def _compute(rdd, split_): + if self.shouldCache: + return env.cacheTracker.getOrCompute(rdd, split_) else: + return rdd.compute(split_) + + if self.checkpoint_path: + if self._checkpoint_rdd is None: + p = os.path.join(self.checkpoint_path, str(split.index)) v = list(self.compute(split)) - with open(p, 'w') as f: - f.write(cPickle.dumps(v)) - return v + with atomic_file(p) as f: + f.write(cPickle.dumps(v, -1)) - if self.shouldCache: - return env.cacheTracker.getOrCompute(self, split) - else: - return self.compute(split) + return v + else: + return _compute(self._checkpoint_rdd, split) + return _compute(self, split) + + def set_rddconf(self, rddconf): + if rddconf is None: + rddconf = dpark.conf.rddconf() + elif rddconf.sort_merge: + num_map = self._num_stream_need() + limit = dpark.conf.MAX_OPEN_FILE + if num_map > limit: + logger.warning("%s: too many split (%d > %d) in parents stages," + " sort merge will use disk", + self.scope.api_callsite, num_map, limit) + rddconf.disk_merge = True + self.rddconf = rddconf def map(self, f): return MappedRDD(self, f) @@ -143,37 +267,38 @@ def filter(self, f): def sample(self, faction, withReplacement=False, seed=12345): return SampleRDD(self, faction, withReplacement, seed) - def union(self, rdd): - return UnionRDD(self.ctx, [self, rdd]) + def union(self, *args): + return UnionRDD(self.ctx, [self] + list(args)) - def sort(self, key=lambda x:x, reverse=False, numSplits=None, taskMemory=None): + def sort(self, key=lambda x: x, reverse=False, numSplits=None, taskMemory=None, rddconf=None): if not len(self): return self if len(self) == 1: return self.mapPartitions(lambda it: sorted(it, key=key, reverse=reverse)) if numSplits is None: numSplits = min(self.ctx.defaultMinSplits, len(self)) - n = numSplits * 10 / len(self) - samples = self.mapPartitions(lambda x:itertools.islice(x, n)).map(key).collect() - keys = sorted(samples, reverse=reverse)[5::10][:numSplits-1] + n = max(numSplits * 10 // len(self), 1) + samples = self.mapPartitions(lambda x: itertools.islice(x, n)).map(key).collect() + keys = sorted(samples, reverse=reverse)[5::10][:numSplits - 1] parter = RangePartitioner(keys, reverse=reverse) aggr = MergeAggregator() - parted = ShuffledRDD(self.map(lambda x:(key(x),x)), aggr, parter, taskMemory).flatMap(lambda (x,y):y) - return parted.mapPartitions(lambda x:sorted(x, key=key, reverse=reverse)) + parted = ShuffledRDD(self.map(lambda x: (key(x), x)), aggr, parter, taskMemory, rddconf=rddconf).flatMap( + lambda x_y: x_y[1]) + return parted.mapPartitions(lambda x: sorted(x, key=key, reverse=reverse)) def glom(self): return GlommedRDD(self) - def cartesian(self, other): - return CartesianRDD(self, other) + def cartesian(self, other, taskMemory=None, cacheMemory=None): + return CartesianRDD(self, other, taskMemory=taskMemory, cacheMemory=cacheMemory) def zipWith(self, other): return ZippedRDD(self.ctx, [self, other]) - def groupBy(self, f, numSplits=None): + def groupBy(self, f, numSplits=None, rddconf=None): if numSplits is None: numSplits = min(self.ctx.defaultMinSplits, len(self)) - return self.map(lambda x: (f(x), x)).groupByKey(numSplits) + return self.map(lambda x: (f(x), x)).groupByKey(numSplits, rddconf=rddconf) def pipe(self, command, quiet=False): if isinstance(command, str): @@ -185,39 +310,55 @@ def fromCsv(self, dialect='excel'): def mapPartitions(self, f): return MapPartitionsRDD(self, f) + mapPartition = mapPartitions def foreach(self, f): def mf(it): for i in it: f(i) + list(self.ctx.runJob(self, mf)) def foreachPartition(self, f): list(self.ctx.runJob(self, f)) def enumeratePartition(self): - return EnumeratePartitionsRDD(self, lambda x,it: itertools.imap(lambda y:(x,y), it)) + return EnumeratePartitionsRDD(self, lambda x, it: map(lambda y: (x, y), it)) def enumerate(self): - return EnumeratePartitionsRDD(self, lambda x,it: - itertools.imap(lambda (y,z):((x,y),z), enumerate(it))) + """ + enumerate this RDD. + + >>> dpark.parallelize(["a", "b", "c", "d"], 3).enumerate().collect() + [(0, 'a'), (1, 'b'), (2, 'c'), (3, 'd')] + """ + starts = [0] + if len(self) > 1: + nums = self.mapPartitions(lambda it: [sum(1 for _ in it)]).collect() + for i in range(len(nums) - 1): + starts.append(starts[-1] + nums[i]) + return EnumeratePartitionsRDD(self, lambda x, it: enumerate(it, starts[x])) def collect(self): - return sum(self.ctx.runJob(self, lambda x:list(x)), []) + return sum(self.ctx.runJob(self, lambda x: list(x)), []) def __iter__(self): - return self.collect() + return chain(self.ctx.runJob(self, lambda x: list(x))) def reduce(self, f): def reducePartition(it): - logger = logging.getLogger(__name__) - if self.err < 1e-8: + logger = get_logger(__name__) + if not self.allow_err: try: return [reduce(f, it)] - except TypeError: - return [] + except TypeError as e: + empty_msg = 'reduce() of empty sequence with no initial value' + if e.message == empty_msg: + return [] + else: + raise e s = None total, err = 0, 0 @@ -228,22 +369,20 @@ def reducePartition(it): s = v else: s = f(s, v) - except Exception, e: + except Exception as e: logger.warning("skip bad record %s: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s" % (float(err)/total)) - if err > total * self.err: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, False) + self.check_err_rate(total, err, True) return [s] if s is not None else [] return reduce(f, chain(self.ctx.runJob(self, reducePartition))) - def uniq(self, numSplits=None, taskMemory=None): - g = self.map(lambda x:(x,None)).reduceByKey(lambda x,y:None, numSplits, taskMemory) - return g.map(lambda (x,y):x) + def uniq(self, numSplits=None, taskMemory=None, rddconf=None): + g = self.map(lambda x: (x, None)).reduceByKey(lambda x, y: None, numSplits, taskMemory, rddconf) + return g.map(lambda x_y1: x_y1[0]) def top(self, n=10, key=None, reverse=False): if reverse: @@ -254,9 +393,9 @@ def topk(it): return heapq.nlargest(n, it, key) return topk(sum(self.ctx.runJob(self, topk), [])) - def hot(self, n=10, numSplits=None, taskMemory=None): - st = self.map(lambda x:(x,1)).reduceByKey(lambda x,y:x+y, numSplits, taskMemory) - return st.top(n, key=lambda x:x[1]) + def hot(self, n=10, numSplits=None, taskMemory=None, rddconf=None): + st = self.map(lambda x: (x, 1)).reduceByKey(lambda x, y: x + y, numSplits, taskMemory, rddconf=rddconf) + return st.top(n, key=lambda x: x[1]) def fold(self, zero, f): '''Aggregate the elements of each partition, and then the @@ -289,14 +428,31 @@ def toList(self): return self.collect() def take(self, n): - if n == 0: return [] + if n == 0: + return [] r = [] p = 0 + n_splits = 1 + last = 0 while len(r) < n and p < len(self): - res = list(self.ctx.runJob(self, lambda x: list(itertools.islice(x, n - len(r))), [p], True))[0] + left = n - len(r) + if p > 0: + # assume we has only 10 but take(11), and get the 10 early, we need to run remaining splits fast. + n_splits = 2 * n_splits + if last > 0: + n_splits = min(int(math.ceil(float(left) * p / len(r))), n_splits) + max_ = min((p + n_splits), len(self)) + splits = list(range(p, max_)) + + logger.info("try to TAKE remaining %d of %d results, from %d/%d splits: [%d, %d)", + left, n, len(splits), len(self), p, max_) + list_list = list(self.ctx.runJob(self, lambda x: list(itertools.islice(x, left)), splits, True)) + res = [item for sublist in list_list for item in sublist][:left] + last = len(res) if res: r.extend(res) - p += 1 + p = max_ + return r def first(self): @@ -306,6 +462,9 @@ def first(self): def saveAsTextFile(self, path, ext='', overwrite=True, compress=False): return OutputTextFileRDD(self, path, ext, overwrite, compress=compress).collect() + def saveAsTFRecordsFile(self, path, ext='', overwrite=True, compress=False): + return OutputTfrecordstFileRDD(self, path, ext, overwrite, compress=compress).collect() + def saveAsTextFileByKey(self, path, ext='', overwrite=True, compress=False): return MultiOutputTextFileRDD(self, path, ext, overwrite, compress=compress).collect() @@ -318,11 +477,23 @@ def saveAsBinaryFile(self, path, fmt, overwrite=True): def saveAsTableFile(self, path, overwrite=True): return OutputTableFileRDD(self, path, overwrite).collect() - def saveAsBeansdb(self, path, depth=0, overwrite=True, compress=True, raw=False): - assert depth<=2, 'only support depth<=2 now' + def saveAsBeansdb(self, path, depth=0, overwrite=True, compress=True, + raw=False, valueWithMeta=False): + """ save (key, value) pair in beansdb format files + + Args: + depth: choice = [0, 1, 2]. + e.g. depth = 2 will write split N to 256 files: + 'path/[0-F]/[0-F]/%03d.data' % N + MUST use depth == 0 to generate data for rivendb + raw: same as in DparkContext.beansdb + valueWithMeta: expect TRIPLE as input value + """ + assert depth <= 2, 'only support depth<=2 now' if len(self) >= 256: - self = self.mergeSplit(len(self) / 256 + 1) - return OutputBeansdbRDD(self, path, depth, overwrite, compress, raw).collect() + self = self.mergeSplit(len(self) // 256 + 1) + return OutputBeansdbRDD(self, path, depth, overwrite, compress, + raw, valueWithMeta).collect() def saveAsTabular(self, path, field_names, **kw): from dpark.tabular import OutputTabularRDD @@ -331,40 +502,102 @@ def saveAsTabular(self, path, field_names, **kw): # Extra functions for (K,V) pairs RDD def reduceByKeyToDriver(self, func): def mergeMaps(m1, m2): - for k,v in m2.iteritems(): - m1[k]=func(m1[k], v) if k in m1 else v + for k, v in six.iteritems(m2): + m1[k] = func(m1[k], v) if k in m1 else v return m1 - return self.map(lambda (x,y):{x:y}).reduce(mergeMaps) - def combineByKey(self, aggregator, splits=None, taskMemory=None): + return self.map(lambda x_y2: {x_y2[0]: x_y2[1]}).reduce(mergeMaps) + + def combineByKey(self, aggregator, splits=None, taskMemory=None, fixSkew=-1, rddconf=None): if splits is None: splits = min(self.ctx.defaultMinSplits, len(self)) if type(splits) is int: - splits = HashPartitioner(splits) - return ShuffledRDD(self, aggregator, splits, taskMemory) - - def reduceByKey(self, func, numSplits=None, taskMemory=None): - aggregator = Aggregator(lambda x:x, func, func) - return self.combineByKey(aggregator, numSplits, taskMemory) - - def groupByKey(self, numSplits=None, taskMemory=None): - createCombiner = lambda x: [x] - mergeValue = lambda x,y:x.append(y) or x - mergeCombiners = lambda x,y: x.extend(y) or x - aggregator = Aggregator(createCombiner, mergeValue, mergeCombiners) - return self.combineByKey(aggregator, numSplits, taskMemory) - - def partitionByKey(self, numSplits=None, taskMemory=None): - return self.groupByKey(numSplits, taskMemory).flatMapValue(lambda x: x) - - def innerJoin(self, other): - o_b = self.ctx.broadcast(other.collectAsMap()) - r = self.filter(lambda (k,v):k in o_b.value).map(lambda (k,v):(k,(v,o_b.value[k]))) - r.mem += (o_b.bytes * 10) >> 20 # memory used by broadcast obj - return r + _thresh = None + if fixSkew > 0 and splits > 1: + _step = 100. / splits + _offsets = [_step * i for i in range(1, splits)] + _percentiles = self.percentiles( + _offsets, sampleRate=fixSkew, func=lambda t: portable_hash(t[0]) + ) + + if _percentiles: + _thresh = [] + for p in _percentiles: + if math.isnan(p): + continue + + p = int(math.ceil(p)) + if not _thresh or p > _thresh[-1]: + _thresh.append(p) + + if len(_thresh) + 1 < splits: + logger.warning('Highly skewed dataset detected!') + + splits = len(_thresh) + 1 + else: + _thresh = None + + splits = HashPartitioner(splits, thresholds=_thresh) + return ShuffledRDD(self, aggregator, splits, taskMemory, rddconf=rddconf) + + def reduceByKey(self, func, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + aggregator = Aggregator(lambda x: x, func, func) + return self.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) + + def groupByKey(self, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + aggregator = GroupByAggregator() + return self.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew, + rddconf=rddconf) + + def topByKey(self, top_n, order_func=None, + reverse=False, num_splits=None, task_memory=None, fixSkew=-1): + """ Base on groupByKey, return the top_n values in each group. + The values in a key are ordered by a function object order_func. + The result values in a key is order in inc order, if you want + dec order, reverse is needed. + We implement the top n function of values in heap. To keep stable in the input values, + we trans the value in the records of rdd into a 4-tuple and sort them on heap. + The values with same return under order function will be sorted by their order added to heap, + only oldest value will be reserve. + After call of combineByKey, we call the map to unpack the value from 4-tuple. + :param top_n: required, limit of values to reserve in each group of a key + :param order_func: optional, order function map a value to a comparable value, default None, when + None, the value itself + :param reverse: optional, bool, when True, the value is sorted in dec order, else inc order + :param num_splits: same with groupByKey + :param task_memory: same with groupByKey + :return: rdd + """ + + # To keep stable in heap, topByKey func need to generate the 4-tuple which is + # in the form of the + # (order_func(v), partition id, sequence id, v), in the end of the combineByKey + # return the v in the 4-th of the tuple + def get_tuple_list(s_id, it, rev): + return [( + i_k_v[1][0], + (order_func(i_k_v[1][1]) if order_func else i_k_v[1][1], + s_id if not rev else -s_id, + i_k_v[0] if not rev else -i_k_v[0], + i_k_v[1][1]) + ) for i_k_v in enumerate(it)] + + aggregator = HeapAggregator(top_n, + order_reverse=reverse) + rdd = EnumeratePartitionsRDD( + self, lambda s_id, it: get_tuple_list(s_id, it, reverse) + ) + return rdd.combineByKey( + aggregator, num_splits, task_memory, fixSkew=fixSkew + ) \ + .map(lambda x_ls: (x_ls[0], sorted(x_ls[1], reverse=reverse))) \ + .map(lambda k_ls: (k_ls[0], [x[-1] for x in k_ls[1]])) + + def partitionByKey(self, numSplits=None, taskMemory=None, rddconf=None): + return self.groupByKey(numSplits, taskMemory, rddconf=rddconf).flatMapValue(lambda x: x) def update(self, other, replace_only=False, numSplits=None, - taskMemory=None): + taskMemory=None, fixSkew=-1, rddconf=None): rdd = self.mapValue( lambda val: (val, 1) # bin('01') for old rdd ).union( @@ -372,11 +605,11 @@ def update(self, other, replace_only=False, numSplits=None, lambda val: (val, 2) # bin('10') for new rdd ) ).reduceByKey( - lambda (val_a, rev_a), (val_b, rev_b): ( - (val_b if rev_b > rev_a else val_a), (rev_a | rev_b) - ), + lambda x, y: (y[0] if y[1] > x[1] else x[0], x[1] | y[1]), numSplits, - taskMemory + taskMemory, + fixSkew=fixSkew, + rddconf=rddconf ) # rev: # 1(01): old value @@ -384,27 +617,55 @@ def update(self, other, replace_only=False, numSplits=None, # 3(11): new updated value if replace_only: rdd = rdd.filter( - lambda (key, (val, rev)): rev != 2 + lambda key_val_rev: key_val_rev[1][1] != 2 ) return rdd.mapValue( - lambda (val, rev): val + lambda val_rev: val_rev[0] ) - def join(self, other, numSplits=None, taskMemory=None): - return self._join(other, (), numSplits, taskMemory) + def innerJoin(self, smallRdd): + """ + This is functionally equivalent to `join`, but `innerJoin` assume `smallRdd` is a + small Data set, and `innerJoin` will broadcast the `smallRdd` to optimize running time. + + >>> x = dpark_context.parallelize([("a", 1), ("b", 4)]) + >>> y = dpark_context.parallelize([("a", 2), ("a", 3)]) + >>> x.innerJoin(y).collect() + [('a', (1, 2)), ('a', (1, 3))] + """ + o = collections.defaultdict(list) + for (k, v) in smallRdd: + o[k].append(v) + o_b = self.ctx.broadcast(o) + + def do_join(k_v): + (k, v) = k_v + for v1 in o_b.value[k]: + yield (k, (v, v1)) + + r = self.flatMap(do_join) + r.mem += (o_b.bytes * 10) >> 20 # memory used by broadcast obj + return r + + def join(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + return self._join(other, (), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) - def leftOuterJoin(self, other, numSplits=None, taskMemory=None): - return self._join(other, (1,), numSplits, taskMemory) + def leftOuterJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + return self._join(other, (1,), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) - def rightOuterJoin(self, other, numSplits=None, taskMemory=None): - return self._join(other, (2,), numSplits, taskMemory) + def rightOuterJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + return self._join(other, (2,), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) - def outerJoin(self, other, numSplits=None, taskMemory=None): - return self._join(other, (1,2), numSplits, taskMemory) + def outerJoin(self, other, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + return self._join(other, (1, 2), numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) - def _join(self, other, keeps, numSplits=None, taskMemory=None): - def dispatch((k,seq)): + def _join(self, other, keeps, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): + + def dispatch(k_seq): + (k, seq) = k_seq vbuf, wbuf = seq + if not isinstance(wbuf, (list, tuple)): + wbuf = list(wbuf) if not vbuf and 2 in keeps: vbuf.append(None) if not wbuf and 1 in keeps: @@ -412,11 +673,13 @@ def dispatch((k,seq)): for vv in vbuf: for ww in wbuf: yield (k, (vv, ww)) - return self.cogroup(other, numSplits, taskMemory).flatMap(dispatch) + + return self.cogroup(other, numSplits, taskMemory, fixSkew=fixSkew, rddconf=rddconf) \ + .flatMap(dispatch) def collectAsMap(self): d = {} - for v in self.ctx.runJob(self, lambda x:list(x)): + for v in self.ctx.runJob(self, lambda x: list(x)): d.update(dict(v)) return d @@ -426,24 +689,64 @@ def mapValue(self, f): def flatMapValue(self, f): return FlatMappedValuesRDD(self, f) - def groupWith(self, others, numSplits=None, taskMemory=None): + def groupWith(self, others, numSplits=None, taskMemory=None, fixSkew=-1, rddconf=None): if isinstance(others, RDD): others = [others] - part = self.partitioner or HashPartitioner(numSplits or self.ctx.defaultParallelism) - return CoGroupedRDD([self]+others, part, taskMemory) + + _numSplits = numSplits + if _numSplits is None: + if self.partitioner is not None: + _numSplits = self.partitioner.numPartitions + else: + _numSplits = self.ctx.defaultParallelism + + _thresh = None + if fixSkew > 0 and _numSplits > 1: + _step = 100. / _numSplits + _offsets = [_step * i for i in range(1, _numSplits)] + _percentiles = self.union(*others) \ + .percentiles( + _offsets, sampleRate=fixSkew, func=lambda t: portable_hash(t[0]) + ) + + if _percentiles: + _thresh = [] + for p in _percentiles: + if math.isnan(p): + continue + + p = int(math.ceil(p)) + if not _thresh or p > _thresh[-1]: + _thresh.append(p) + + if len(_thresh) + 1 < _numSplits: + logger.warning('Highly skewed dataset detected!') + + _numSplits = len(_thresh) + 1 + else: + _thresh = None + + part = HashPartitioner(_numSplits, thresholds=_thresh) + rdd = CoGroupedRDD([self] + others, part, taskMemory, rddconf=rddconf) + return rdd cogroup = groupWith def lookup(self, key): if self.partitioner: index = self.partitioner.getPartition(key) + def process(it): - for k,v in it: + for k, v in it: if k == key: return v - return list(self.ctx.runJob(self, process, [index], False))[0] + + result = list(self.ctx.runJob(self, process, [index], False)) + return result[0] if result else None else: - raise Exception("lookup() called on an RDD without a partitioner") + logger.warning("Too much time may be taken to lookup in a RDD without a partitioner!") + result = self.flatMap(lambda k_v: [k_v[1]] if k_v[0] == key else []).take(1) + return result[0] if result else None def asTable(self, fields, name=''): from dpark.table import TableRDD @@ -463,22 +766,112 @@ def _batch(iterable): def adcount(self): "approximate distinct counting" - r = self.map(lambda x:(1, x)).adcountByKey(1).collectAsMap() + r = self.map(lambda x: (1, x)).adcountByKey(1).collectAsMap() return r and r[1] or 0 - def adcountByKey(self, splits=None, taskMemory=None): + def adcountByKey(self, splits=None, taskMemory=None, fixSkew=-1): try: from pyhll import HyperLogLog except ImportError: - from hyperloglog import HyperLogLog + from dpark.utils.hyperloglog import HyperLogLog + def create(v): return HyperLogLog([v], 16) + def combine(s, v): return s.add(v) or s + def merge(s1, s2): return s1.update(s2) or s1 + agg = Aggregator(create, combine, merge) - return self.combineByKey(agg, splits, taskMemory).mapValue(len) + return self.combineByKey(agg, splits, taskMemory, fixSkew=fixSkew) \ + .mapValue(len) + + def percentiles(self, p, sampleRate=1.0, func=None): + def _(it): + from dpark.utils.tdigest import TDigest + digest = TDigest() + for k in it: + digest.add(k) + + digest.compress() + yield digest + + if sampleRate <= 0: + raise ValueError('Sample Rate should be positive.') + + if sampleRate >= 1.0: + rdd = self + else: + rdd = self.sample(sampleRate) + + if func: + rdd = rdd.map(func) + + _digest = rdd.mapPartitions(_).reduce(lambda x, y: x + y) + _digest.compress() + return [_digest.quantile(pp / 100.) for pp in p] + + def percentilesByKey(self, p, sampleRate=1.0, func=None, + numSplits=None, taskMemory=None, fixSkew=-1): + def _create(x): + from dpark.utils.tdigest import TDigest + digest = TDigest() + digest.add(x) + return digest + + def _update(d, x): + d.add(x) + return d + + def _merge(d1, d2): + d = d1 + d2 + d.compress() + return d + + def _(d): + return [d.quantile(pp / 100.) for pp in p] + + if sampleRate <= 0: + raise ValueError('Sample Rate should be positive.') + + if sampleRate >= 1.0: + rdd = self + else: + rdd = self.sample(sampleRate) + + if func: + rdd = rdd.mapValue(func) + + aggregator = Aggregator(_create, _update, _merge) + return rdd.combineByKey(aggregator, numSplits, taskMemory, fixSkew=fixSkew) \ + .mapValue(_) + + def with_cpus(self, cpus): + self.cpus = cpus + return self + + def with_gpus(self, gpus): + self.gpus = gpus + return self + + def with_mem(self, mem): + self.mem = mem + return self + + def check_err_rate(self, total, err, end=False): + if not self.exc_info: + self.exc_info = sys.exc_info() + threshold = total * self.err_ratio + if not end: + fail = total > 100 and err > threshold * 10 + else: + fail = err > threshold + if fail: + msg_tmpl = "too many error occured: %s/%s=%s, %s" + msg = msg_tmpl % (err, total, float(err) / total, self.exc_info[1]) + raise (DparkUserFatalError, DparkUserFatalError(msg), self.exc_info[2]) class DerivedRDD(RDD): @@ -486,29 +879,39 @@ def __init__(self, rdd): RDD.__init__(self, rdd.ctx) self.prev = rdd self.mem = max(self.mem, rdd.mem) - self.dependencies = [OneToOneDependency(rdd)] + self.cpus = rdd.cpus + self.gpus = rdd.gpus + self._dependencies = [OneToOneDependency(rdd)] + self._splits = self.prev.splits + self._preferred_locs = self.prev._preferred_locs + self.repr_name = '<%s %s>' % (self.__class__.__name__, rdd) + self.lineage += rdd.lineage - def __len__(self): - return len(self.prev) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.prev = None - def __repr__(self): - return '<%s %s>' % (self.__class__.__name__, self.prev) + def num_stream(self): + return self.prev.num_stream() @property def splits(self): - return self.prev.splits - - def _preferredLocations(self, split): - return self.prev.preferredLocations(split) + if self._checkpoint_rdd: + return self._checkpoint_rdd.splits + return self._splits class MappedRDD(DerivedRDD): - def __init__(self, prev, func=lambda x:x): + def __init__(self, prev, func=lambda x: x): DerivedRDD.__init__(self, prev) self.func = func + @property + def params(self): + return func_info(self.func) + def compute(self, split): - if self.err < 1e-8: + if not self.allow_err: return (self.func(v) for v in self.prev.iterator(split)) return self._compute_with_error(split) @@ -518,14 +921,12 @@ def _compute_with_error(self, split): try: total += 1 yield self.func(v) - except Exception, e: + except Exception as e: logger.warning("ignored record %r: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, False) - if err > total * self.err: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, True) @cached def __getstate__(self): @@ -538,12 +939,12 @@ def __setstate__(self, state): try: self.func = load_func(code) except Exception: - print 'load failed', self.__class__, code[:1024] raise + class FlatMappedRDD(MappedRDD): def compute(self, split): - if self.err < 1e-8: + if not self.allow_err: return chain(self.func(v) for v in self.prev.iterator(split)) return self._compute_with_error(split) @@ -554,19 +955,16 @@ def _compute_with_error(self, split): total += 1 for k in self.func(v): yield k - except Exception, e: + except Exception as e: logger.warning("ignored record %r: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s, %s" % ((float(err)/total), e)) - - if err > total * self.err: - raise Exception("too many error occured: %s, %s" % ((float(err)/total), e)) + self.check_err_rate(total, err, False) + self.check_err_rate(total, err, True) class FilteredRDD(MappedRDD): def compute(self, split): - if self.err < 1e-8: + if not self.allow_err: return (v for v in self.prev.iterator(split) if self.func(v)) return self._compute_with_error(split) @@ -577,44 +975,43 @@ def _compute_with_error(self, split): total += 1 if self.func(v): yield v - except Exception, e: + except Exception as e: logger.warning("ignored record %r: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, False) + self.check_err_rate(total, err, True) - if err > total * self.err: - raise Exception("too many error occured: %s" % (float(err)/total)) class GlommedRDD(DerivedRDD): def compute(self, split): yield list(self.prev.iterator(split)) + class MapPartitionsRDD(MappedRDD): def compute(self, split): return self.func(self.prev.iterator(split)) + class EnumeratePartitionsRDD(MappedRDD): def compute(self, split): return self.func(split.index, self.prev.iterator(split)) + class PipedRDD(DerivedRDD): def __init__(self, prev, command, quiet=False, shell=False): DerivedRDD.__init__(self, prev) self.command = command self.quiet = quiet self.shell = shell - - def __repr__(self): - return '' % (' '.join(self.command), self.prev) + self.repr_name = '' % (' '.join(command), prev) def compute(self, split): import subprocess devnull = open(os.devnull, 'w') p = subprocess.Popen(self.command, stdin=subprocess.PIPE, - stdout=subprocess.PIPE, - stderr=self.quiet and devnull or sys.stderr, - shell=self.shell) + stdout=subprocess.PIPE, + stderr=self.quiet and devnull or sys.stderr, + shell=self.shell) def read(stdin): try: @@ -625,14 +1022,14 @@ def read(stdin): else: return try: - if isinstance(first, str) and first.endswith('\n'): + if isinstance(first, six.binary_type) and first.endswith(b'\n'): stdin.write(first) stdin.writelines(it) else: - stdin.write("%s\n"%first) - stdin.writelines("%s\n"%x for x in it) - except Exception, e: - if not (isinstance(e, IOError) and e.errno == 32): # Broken Pipe + stdin.write(("%s\n" % first).encode('utf-8')) + stdin.writelines(("%s\n" % x).encode('utf-8') for x in it) + except Exception as e: + if not (isinstance(e, IOError) and e.errno == 32): # Broken Pipe self.error = e p.kill() finally: @@ -649,9 +1046,11 @@ def read(self, p): if self.error: raise self.error ret = p.wait() - #if ret: + p.stdout.close() + # if ret: # raise Exception('Subprocess exited with status %d' % ret) + class MappedValuesRDD(MappedRDD): @property def partitioner(self): @@ -659,80 +1058,95 @@ def partitioner(self): def compute(self, split): func = self.func - if self.err < 1e-8: - return ((k,func(v)) for k,v in self.prev.iterator(split)) + if not self.allow_err: + return ((k, func(v)) for k, v in self.prev.iterator(split)) return self._compute_with_error(split) def _compute_with_error(self, split): func = self.func total, err = 0, 0 - for k,v in self.prev.iterator(split): + for k, v in self.prev.iterator(split): try: total += 1 - yield (k,func(v)) - except Exception, e: + yield (k, func(v)) + except Exception as e: logger.warning("ignored record %r: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s" % (float(err)/total)) - if err > total * self.err: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, False) + self.check_err_rate(total, err, True) + class FlatMappedValuesRDD(MappedValuesRDD): def compute(self, split): total, err = 0, 0 - for k,v in self.prev.iterator(split): + for k, v in self.prev.iterator(split): try: total += 1 for vv in self.func(v): - yield k,vv - except Exception, e: + yield k, vv + except Exception as e: logger.warning("ignored record %r: %s", v, e) err += 1 - if total > 100 and err > total * self.err * 10: - raise Exception("too many error occured: %s" % (float(err)/total)) - if err > total * self.err: - raise Exception("too many error occured: %s" % (float(err)/total)) + self.check_err_rate(total, err, False) + self.check_err_rate(total, err, True) + class ShuffledRDDSplit(Split): def __hash__(self): return self.index + class ShuffledRDD(RDD): - def __init__(self, parent, aggregator, part, taskMemory=None): + def __init__(self, parent, aggregator, part, taskMemory=None, rddconf=None): RDD.__init__(self, parent.ctx) - self.parent = parent - self.numParts = len(parent) self.aggregator = aggregator self._partitioner = part if taskMemory: self.mem = taskMemory self._splits = [ShuffledRDDSplit(i) for i in range(part.numPartitions)] + self._parent_length = len(parent) self.shuffleId = self.ctx.newShuffleId() - self.dependencies = [ShuffleDependency(self.shuffleId, - parent, aggregator, part)] - self.name = '' % self.parent + self.set_rddconf(rddconf) + self._dependencies = [ShuffleDependency(self.shuffleId, + parent, aggregator, part, rddconf=self.rddconf)] + self.repr_name = '' % parent + if isinstance(aggregator, GroupByAggregator): + self.rddconf.op = 'groupby' - def __len__(self): - return self.partitioner.numPartitions - - def __repr__(self): - return self.name + self.lineage += parent.lineage @cached def __getstate__(self): d = RDD.__getstate__(self) - d.pop('parent', None) return d def compute(self, split): - merger = Merger(self.numParts, self.aggregator.mergeCombiners) - fetcher = env.shuffleFetcher - fetcher.fetch(self.shuffleId, split.index, merger.merge) + merger = Merger.get(self.rddconf, aggregator=self.aggregator, size=0, api_callsite=self.scope.api_callsite) + if self.rddconf.sort_merge: + fetcher = SortShuffleFetcher() + iters = fetcher.get_iters(self.shuffleId, split.index) + merger.merge(iters) + else: + fetcher = env.shuffleFetcher + fetcher.fetch(self.shuffleId, split.index, merger.merge) return merger + def num_stream(self): + if self.rddconf.sort_merge: + if self.rddconf.disk_merge: + return 100 + else: + return self._num_stream_need() + return 0 + + def _num_stream_need(self): + return self._parent_length + + +DEFAULT_CACHE_MEMORY_SIZE = 256 + class CartesianSplit(Split): def __init__(self, idx, s1, s2): @@ -740,104 +1154,226 @@ def __init__(self, idx, s1, s2): self.s1 = s1 self.s2 = s2 + class CartesianRDD(RDD): - def __init__(self, rdd1, rdd2): + def __init__(self, rdd1, rdd2, taskMemory=None, cacheMemory=DEFAULT_CACHE_MEMORY_SIZE): RDD.__init__(self, rdd1.ctx) self.rdd1 = rdd1 self.rdd2 = rdd2 - self.mem = max(rdd1.mem, rdd2.mem) * 1.5 - self.numSplitsInRdd2 = n = len(rdd2) - self._splits = [CartesianSplit(s1.index*n+s2.index, s1, s2) - for s1 in rdd1.splits for s2 in rdd2.splits] - self.dependencies = [CartesianDependency(rdd1, True, n), - CartesianDependency(rdd2, False, n)] + self.cache_memory = int(max( + DEFAULT_CACHE_MEMORY_SIZE, + cacheMemory or DEFAULT_CACHE_MEMORY_SIZE + )) + self.mem = int(max( + taskMemory or self.ctx.options.mem, + rdd1.mem * 1.5, + rdd2.mem * 1.5, + self.cache_memory * 2.5 + )) + self.cpus = max( + rdd1.cpus, + rdd2.cpus + ) + self.gpus = int(max( + rdd1.gpus, + rdd2.gpus + )) - def __len__(self): - return len(self.rdd1) * len(self.rdd2) + self.numSplitsInRdd2 = n = len(rdd2) + self._splits = [CartesianSplit(s1.index * n + s2.index, s1, s2) + for s1 in rdd1.splits for s2 in rdd2.splits] + self._dependencies = [CartesianDependency(rdd1, True, n), + CartesianDependency(rdd2, False, n)] + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = rdd1.preferredLocations(split.s1) + rdd2.preferredLocations(split.s2) - def __repr__(self): - return '' % (self.rdd1, self.rdd2) + self.repr_name = '' % (self.rdd1, self.rdd2) + self.lineage += rdd1.lineage + rdd2.lineage - def _preferredLocations(self, split): - return self.rdd1.preferredLocations(split.s1) + self.rdd2.preferredLocations(split.s2) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdd1 = self.rdd2 = None def compute(self, split): - b = None - for i in self.rdd1.iterator(split.s1): - if b is None: - b = [] - for j in self.rdd2.iterator(split.s2): - yield (i, j) - b.append(j) - else: - for j in b: - yield (i,j) + saved = False + _cached = None + basedir = env.workdir.alloc_tmp_dir("cartesian") + with tempfile.SpooledTemporaryFile(self.cache_memory << 20, dir=basedir) as f: + for i in self.rdd1.iterator(split.s1): + if not saved: + with gzip.GzipFile('dummy', fileobj=f, compresslevel=1) as gf: + pickler = Pickler(gf, -1) + for j in self.rdd2.iterator(split.s2): + yield i, j + pickler.dump(j) + + saved = True + + elif _cached is not None: + # speedup when memory is enough + for j in _cached: + yield i, j + + else: + if not f._rolled: + _cached = [] + + f.seek(0) + with gzip.GzipFile(fileobj=f, mode='rb') as gf: + unpickler = Unpickler(gf) + try: + while True: + j = unpickler.load() + yield i, j + if _cached is not None: + _cached.append(j) + except EOFError: + pass + + if _cached is not None: + f.close() + + def num_stream(self): + return self.rdd1.num_stream() + self.rdd2.num_stream() + class CoGroupSplitDep: pass + + class NarrowCoGroupSplitDep(CoGroupSplitDep): def __init__(self, rdd, split): self.rdd = rdd self.split = split + + class ShuffleCoGroupSplitDep(CoGroupSplitDep): def __init__(self, shuffleId): self.shuffleId = shuffleId + class CoGroupSplit(Split): def __init__(self, idx, deps): self.index = idx self.deps = deps + def __hash__(self): return self.index -class CoGroupAggregator: - def createCombiner(self, v): - return [v] - def mergeValue(self, c, v): - return c + [v] - def mergeCombiners(self, c, v): - return c + v class CoGroupedRDD(RDD): - def __init__(self, rdds, partitioner, taskMemory=None): + + def __init__(self, rdds, partitioner, taskMemory=None, rddconf=None): RDD.__init__(self, rdds[0].ctx) - self.len = len(rdds) + self.size = len(rdds) if taskMemory: self.mem = taskMemory - self.aggregator = CoGroupAggregator() + self.aggregator = GroupByAggregator() self._partitioner = partitioner - self.dependencies = dep = [rdd.partitioner == partitioner - and OneToOneDependency(rdd) - or ShuffleDependency(self.ctx.newShuffleId(), - rdd, self.aggregator, partitioner) - for i,rdd in enumerate(rdds)] - self._splits = [CoGroupSplit(j, - [isinstance(dep[i],ShuffleDependency) - and ShuffleCoGroupSplitDep(dep[i].shuffleId) - or NarrowCoGroupSplitDep(r, r.splits[j]) - for i,r in enumerate(rdds)]) - for j in range(partitioner.numPartitions)] - self.name = ('' % (','.join(str(rdd) for rdd in rdds)))[:80] + self.repr_name = ('' % (','.join(str(rdd) for rdd in rdds)))[:80] + self._preferred_locs = {} + self.set_rddconf(rddconf) + self.rddconf.op = dpark.conf.OP_COGROUP + for rdd in rdds: + self.lineage += rdd.lineage + + def _get_rdd_deps(): + return [rdd.partitioner == partitioner + and OneToOneDependency(rdd) + or ShuffleDependency(self.ctx.newShuffleId(), + rdd, self.aggregator, partitioner, rddconf=self.rddconf) + for i, rdd in enumerate(rdds)] + + self._dependencies = deps = _get_rdd_deps() + + def _get_split_deps(j): + return [rdd.partitioner == partitioner + and NarrowCoGroupSplitDep(rdd, rdd.splits[j]) + or ShuffleCoGroupSplitDep(deps[i].shuffleId) + for i, rdd in enumerate(rdds)] + + self._splits = [CoGroupSplit(j, _get_split_deps(j)) for j in range(partitioner.numPartitions)] + for split in self._splits: + self._preferred_locs[split] = sum([dep.rdd.preferredLocations(dep.split) for dep in split.deps + if isinstance(dep, NarrowCoGroupSplitDep)], []) + + def _compute_hash_merge(self, split, merger): + for i, dep in enumerate(split.deps): + if isinstance(dep, NarrowCoGroupSplitDep): + merger.merge(dep.rdd.iterator(dep.split), -1, i) + elif isinstance(dep, ShuffleCoGroupSplitDep): - def __len__(self): - return self.partitioner.numPartitions + def merge(items, map_id): + merger.merge(items, map_id, i) - def __repr__(self): - return self.name + env.shuffleFetcher.fetch(dep.shuffleId, split.index, merge) - def _preferredLocations(self, split): - return sum([dep.rdd.preferredLocations(dep.split) for dep in split.deps - if isinstance(dep, NarrowCoGroupSplitDep)], []) + def _compute_sort_merge(self, split, merger): - def compute(self, split): - m = CoGroupMerger(self.len) - for i,dep in enumerate(split.deps): + def _enum_value(items, n): + for k, v in items: + yield k, (n, v) + + iters = [] + fetcher = SortShuffleFetcher() + for i, dep in enumerate(split.deps): if isinstance(dep, NarrowCoGroupSplitDep): - m.append(i, dep.rdd.iterator(dep.split)) + it = sorted(dep.rdd.iterator(dep.split), key=lambda x: x[0]) + it = self.aggregator.aggregate_sorted(it) + it = _enum_value(it, i) + iters.append(it) elif isinstance(dep, ShuffleCoGroupSplitDep): - def merge(items): - m.extend(i, items) - env.shuffleFetcher.fetch(dep.shuffleId, split.index, merge) - return m + its = fetcher.get_iters(dep.shuffleId, split.index) + iters.extend([_enum_value(it, i) for it in its]) + merger.merge(iters) + + def _compute_sort_merge_iter(self, split, merger): + iters = [] + fetcher = SortShuffleFetcher() + for i, dep in enumerate(split.deps): + if isinstance(dep, NarrowCoGroupSplitDep): + it = sorted(dep.rdd.iterator(dep.split), key=lambda x: x[0]) + it = self.aggregator.aggregate_sorted(it) + iters.append(it) + elif isinstance(dep, ShuffleCoGroupSplitDep): + its = fetcher.get_iters(dep.shuffleId, split.index) + rddconf = self.rddconf.dup(op=dpark.conf.OP_GROUPBY) + m = Merger.get(rddconf, size=self.size, api_callsite=self.scope.api_callsite) + m.merge(its) + iters.append(m) + merger.merge(iters) + + def compute(self, split): + rddconf = self.rddconf + merger = Merger.get(rddconf, size=self.size, api_callsite=self.scope.api_callsite) + if rddconf.sort_merge: + if rddconf.iter_group: + self._compute_sort_merge_iter(split, merger) + else: + self._compute_sort_merge(split, merger) + else: + self._compute_hash_merge(split, merger) + return merger + + def num_stream(self): + rddconf = self.rddconf + if rddconf.sort_merge: + if rddconf.disk_merge: + return sum([d.rdd.num_stream() + for d in self._dependencies + if isinstance(d, NarrowCoGroupSplitDep)]) + else: + return self._num_stream_need() + return 0 + + def _num_stream_need(self): + num_map = 0 + for d in self._dependencies: + if isinstance(d, ShuffleDependency): + num_map += len(d.rdd) + else: + num_map += d.rdd.num_stream() + return num_map class SampleRDD(DerivedRDD): @@ -846,16 +1382,14 @@ def __init__(self, prev, frac, withReplacement, seed): self.frac = frac self.withReplacement = withReplacement self.seed = seed - - def __repr__(self): - return '' % (self.frac, self.prev) + self.repr_name = '' % (frac, prev) def compute(self, split): rd = random.Random(self.seed + split.index) if self.withReplacement: olddata = list(self.prev.iterator(split)) sampleSize = int(math.ceil(len(olddata) * self.frac)) - for i in xrange(sampleSize): + for i in range(sampleSize): yield rd.choice(olddata) else: for i in self.prev.iterator(split): @@ -869,46 +1403,57 @@ def __init__(self, idx, rdd, split): self.rdd = rdd self.split = split + class UnionRDD(RDD): + def __init__(self, ctx, rdds): RDD.__init__(self, ctx) - self.mem = rdds[0].mem if rdds else self.mem + if rdds: + self.mem = max(rdd.mem for rdd in rdds) + self.cpus = max(rdd.cpus for rdd in rdds) + self.gpus = max(rdd.cpus for rdd in rdds) + pos = 0 for rdd in rdds: self._splits.extend([UnionSplit(pos + i, rdd, sp) for i, sp in enumerate(rdd.splits)]) - self.dependencies.append(RangeDependency(rdd, 0, pos, len(rdd))) + self._dependencies.append(RangeDependency(rdd, 0, pos, len(rdd))) pos += len(rdd) - self.name = '' % (len(rdds), ','.join(str(rdd) for rdd in rdds[:1])) - - def __repr__(self): - return self.name - - def _preferredLocations(self, split): - return split.rdd.preferredLocations(split.split) + self.repr_name = '' % (len(rdds), ','.join(str(rdd) for rdd in rdds[:1])) + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = split.rdd.preferredLocations(split.split) def compute(self, split): return split.rdd.iterator(split.split) + @property + def ui_label(self): + return "{}[{}]({})".format(self.__class__.__name__, len(self), len(self._dependencies)) + + class SliceRDD(RDD): def __init__(self, rdd, i, j): RDD.__init__(self, rdd.ctx) self.rdd = rdd self.mem = rdd.mem + self.cpus = rdd.cpus + self.gpus = rdd.gpus if j > len(rdd): j = len(rdd) self.i = i self.j = j self._splits = rdd.splits[i:j] - self.dependencies = [RangeDependency(rdd, i, 0, j-i)] - - def __len__(self): - return self.j - self.i + self._dependencies = [RangeDependency(rdd, i, 0, j - i)] + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = rdd.preferredLocations(split) - def __repr__(self): - return '' % (self.i, self.j, self.rdd) + self.repr_name = '' % (i, j, rdd) + self.lineage += rdd.lineage - def _preferredLocations(self, split): - return self.rdd.preferredLocations(split) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdd = None def compute(self, split): return self.rdd.iterator(split) @@ -919,30 +1464,34 @@ def __init__(self, index, splits): self.index = index self.splits = splits + class MergedRDD(RDD): def __init__(self, rdd, splitSize=None, numSplits=None): RDD.__init__(self, rdd.ctx) if splitSize is None: - splitSize = (len(rdd) + numSplits - 1) / numSplits - numSplits = (len(rdd) + splitSize - 1) / splitSize + splitSize = (len(rdd) + numSplits - 1) // numSplits + numSplits = (len(rdd) + splitSize - 1) // splitSize self.rdd = rdd self.mem = rdd.mem + self.cpus = rdd.cpus + self.gpus = rdd.gpus self.splitSize = splitSize self.numSplits = numSplits splits = rdd.splits - self._splits = [MultiSplit(i, splits[i*splitSize:(i+1)*splitSize]) - for i in range(numSplits)] - self.dependencies = [OneToRangeDependency(rdd, splitSize, len(rdd))] + self._splits = [MultiSplit(i, splits[i * splitSize:(i + 1) * splitSize]) + for i in range(numSplits)] + self._dependencies = [OneToRangeDependency(rdd, splitSize, len(rdd))] + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = sum([rdd.preferredLocations(sp) for sp in split.splits], []) - def __len__(self): - return self.numSplits - - def __repr__(self): - return '' % (self.splitSize, self.rdd) + self.repr_name = '' % (splitSize, rdd) + self.lineage += rdd.lineage - def _preferredLocations(self, split): - return sum([self.rdd.preferredLocations(sp) for sp in split.splits], []) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdd = None def compute(self, split): return chain(self.rdd.iterator(sp) for sp in split.splits) @@ -954,41 +1503,55 @@ def __init__(self, ctx, rdds): RDD.__init__(self, ctx) self.rdds = rdds self.mem = max(r.mem for r in rdds) + self.cpus = max(r.cpus for r in rdds) + self.gpus = max(r.gpus for r in rdds) self._splits = [MultiSplit(i, splits) - for i, splits in enumerate(zip(*[rdd.splits for rdd in rdds]))] - self.dependencies = [OneToOneDependency(rdd) for rdd in rdds] - - def __len__(self): - return len(self.rdds[0]) - - def __repr__(self): - return '' % (','.join(str(rdd) for rdd in self.rdds)) + for i, splits in enumerate(zip(*[rdd.splits for rdd in rdds]))] + self._dependencies = [OneToOneDependency(rdd) for rdd in rdds] + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = sum( + [rdd.preferredLocations(sp) for rdd, sp in zip(self.rdds, split.splits)], []) + self.repr_name = '' % (','.join(str(rdd) for rdd in rdds)) + for rdd in rdds: + self.lineage += rdd.lineage - def _preferredLocations(self, split): - return sum([rdd.preferredLocations(sp) - for rdd,sp in zip(self.rdds, split.splits)], []) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdds = [] def compute(self, split): - return itertools.izip(*[rdd.iterator(sp) - for rdd, sp in zip(self.rdds, split.splits)]) + return zip( + *[rdd.iterator(sp) for rdd, sp in zip(self.rdds, split.splits)] + ) + + def num_stream(self): + return sum([rdd.num_stream() for rdd in self.rdds]) class CSVReaderRDD(DerivedRDD): def __init__(self, prev, dialect='excel'): DerivedRDD.__init__(self, prev) self.dialect = dialect - - def __repr__(self): - return '' % (self.dialect, self.prev) + self.repr_name = '' % (dialect, prev) def compute(self, split): return csv.reader(self.prev.iterator(split), self.dialect) class ParallelCollectionSplit: - def __init__(self, index, values): + def __init__(self, ctx, index, values): self.index = index - self.values = values + _values = cPickle.dumps(values, -1) + length = len(_values) + data_limit = ctx.data_limit + if data_limit is None or length < data_limit: + self.values = _values + self.is_broadcast = False + else: + self.values = ctx.broadcast(_values) + self.is_broadcast = True + class ParallelCollection(RDD): def __init__(self, ctx, data, numSlices, taskMemory=None): @@ -997,15 +1560,18 @@ def __init__(self, ctx, data, numSlices, taskMemory=None): if taskMemory: self.mem = taskMemory slices = self.slice(data, max(1, min(self.size, numSlices))) - self._splits = [ParallelCollectionSplit(i, slices[i]) - for i in range(len(slices))] - self.dependencies = [] - - def __repr__(self): - return '' % self.size + self._splits = [ParallelCollectionSplit(ctx, i, slices[i]) + for i in range(len(slices))] + self._dependencies = [] + self.repr_name = '' % self.size def compute(self, split): - return split.values + if split.is_broadcast: + _values = split.values.value + else: + _values = split.values + + return cPickle.loads(_values) @classmethod def slice(cls, data, numSlices): @@ -1014,24 +1580,48 @@ def slice(cls, data, numSlices): m = len(data) if not m: return [[]] - n = m / numSlices + n = m // numSlices if m % numSlices != 0: n += 1 - if isinstance(data, xrange): + if isinstance(data, range): first = data[0] - last = data[m-1] - step = (last - first) / (m-1) + last = data[m - 1] + step = (last - first) // (m - 1) nstep = step * n - slices = [xrange(first+i*nstep, first+(i+1)*nstep, step) - for i in range(numSlices-1)] - slices.append(xrange(first+(numSlices-1)*nstep, - min(last+step, first+numSlices*nstep), step)) + slices = [range(first + i * nstep, first + (i + 1) * nstep, step) + for i in range(numSlices - 1)] + slices.append(range(first + (numSlices - 1) * nstep, + min(last + step, first + numSlices * nstep), step)) return slices if not isinstance(data, list): data = list(data) - return [data[i*n : i*n+n] for i in range(numSlices)] + return [data[i * n: i * n + n] for i in range(numSlices)] +class CheckpointRDD(RDD): + def __init__(self, ctx, path): + RDD.__init__(self, ctx) + self.path = path + files = self.generated_files(path) + if not files or files[0] != '0' or files[-1] != str(len(files) - 1): + raise RuntimeError('Invalid checkpoint directory: %s' % path) + + self.files = files + self._splits = [Split(i) for i in range(len(files))] + + @classmethod + def generated_files(cls, path): + return sorted(filter(str.isdigit, os.listdir(path)), key=int) + + def compute(self, split): + try: + with open(os.path.join(self.path, self.files[split.index]), 'rb') as f: + return cPickle.loads(f.read()) + except IOError: + time.sleep(1) + with open(os.path.join(self.path, self.files[split.index]), 'rb') as f: + return cPickle.loads(f.read()) + class PartialSplit(Split): def __init__(self, index, begin, end): @@ -1039,99 +1629,159 @@ def __init__(self, index, begin, end): self.begin = begin self.end = end -class TextFileRDD(RDD): - DEFAULT_SPLIT_SIZE = 64*1024*1024 +class TextFileRDD(RDD): + DEFAULT_SPLIT_SIZE = 64 * 1024 * 1024 def __init__(self, ctx, path, numSplits=None, splitSize=None): RDD.__init__(self, ctx) self.path = path - self.fileinfo = moosefs.open_file(path) - self.size = size = self.fileinfo.length if self.fileinfo else os.path.getsize(path) + with closing(open_file(path)) as file_: + self.size = size = file_.length - if splitSize is None: - if numSplits is None: - splitSize = self.DEFAULT_SPLIT_SIZE - else: - splitSize = size / numSplits or self.DEFAULT_SPLIT_SIZE - n = size / splitSize - if size % splitSize > 0: - n += 1 - self.splitSize = splitSize - self.len = n - self._splits = [PartialSplit(i, i*splitSize, min(size, (i+1) * splitSize)) - for i in range(self.len)] - - def __len__(self): - return self.len - - def __repr__(self): - return '<%s %s>' % (self.__class__.__name__, self.path) + if splitSize is None: + if numSplits is None: + splitSize = self.DEFAULT_SPLIT_SIZE + else: + splitSize = size // numSplits or self.DEFAULT_SPLIT_SIZE + numSplits = size // splitSize + if size % splitSize > 0: + numSplits += 1 + self.splitSize = splitSize + self._splits = [PartialSplit(i, i * splitSize, min(size, (i + 1) * splitSize)) + for i in range(numSplits)] + + self._preferred_locs = {} + for split in self._splits: + if self.splitSize != CHUNKSIZE: + start = split.begin // CHUNKSIZE + end = (split.end + CHUNKSIZE - 1) // CHUNKSIZE + self._preferred_locs[split] = sum((file_.locs(i) for i in range(start, end)), []) + else: + self._preferred_locs[split] = file_.locs(split.begin // self.splitSize) + hostnames = [] + for loc in self._preferred_locs[split]: + host = loc + try: + host = socket.gethostbyaddr(loc)[0] + except IOError as e: + logger.warning('get hostname exec %s for loc %s', e.message, loc) + hostnames.append(host) + self._preferred_locs[split] = hostnames + self.repr_name = '<%s %s>' % (self.__class__.__name__, path) - def _preferredLocations(self, split): - if not self.fileinfo: - return [] - if self.splitSize != moosefs.CHUNKSIZE: - start = split.begin / moosefs.CHUNKSIZE - end = (split.end + moosefs.CHUNKSIZE - 1)/ moosefs.CHUNKSIZE - return sum((self.fileinfo.locs(i) for i in range(start, end)), []) - else: - return self.fileinfo.locs(split.begin / self.splitSize) + @property + def params(self): + return self.path def open_file(self): - if self.fileinfo: - return moosefs.ReadableFile(self.fileinfo) - else: - return open(self.path, 'r', 4096 * 1024) + return open_file(self.path) def compute(self, split): - f = self.open_file() - #if len(self) == 1 and split.index == 0 and split.begin == 0: - # return f - - start = split.begin - end = split.end - if start > 0: - f.seek(start-1) - byte = f.read(1) - while byte != '\n': + with closing(self.open_file()) as f: + start = split.begin + end = split.end + if start > 0: + f.seek(start - 1) byte = f.read(1) - if not byte: - return [] - start += 1 + while byte != b'\n': + byte = f.read(1) + if not byte: + return + start += 1 - if start >= end: - return [] - - #if self.fileinfo: - # # cut by end - # if end < self.fileinfo.length: - # f.seek(end-1) - # while f.read(1) not in ('', '\n'): - # end += 1 - # f.length = end - # f.seek(start) - # return f + if start >= end: + return - return self.read(f, start, end) + for l in self.read(f, start, end): + yield l def read(self, f, start, end): for line in f: + size = len(line) + if not six.PY2: + line = line.decode('utf-8') + if line.endswith('\n'): yield line[:-1] else: yield line - start += len(line) + start += size if start >= end: break - f.close() + + +class TfrecordsRDD(TextFileRDD): + DEFAULT_READ_SIZE = 1 << 10 + + def __init__(self, ctx, path, numSplits=None, splitSize=None): + TextFileRDD.__init__(self, ctx, path, numSplits, splitSize) + + def compute(self, split): + with closing(self.open_file()) as f: + start = split.begin + end = split.end + if start > 0: + # modified below + f.seek(start) + buffer = f.read(min(self.DEFAULT_READ_SIZE, end - f.tell())) + while start < end: + cursor = 0 + while cursor < len(buffer) - 11 and not self.check_split_point(buffer[cursor:cursor + 12]): + cursor += 1 + start += cursor + if cursor == len(buffer) - 11: + start += 11 + buffer = buffer[-11:] + f.read(min(self.DEFAULT_READ_SIZE, end - f.tell())) + else: + break + + if start >= end: + return + + f.seek(start) + while start < end: + record = self.get_single_record(f) + yield record + start += len(record) + 16 + + def check_split_point(self, buf): + buf_length_expected = 12 + if not buf: + return False + if len(buf) != buf_length_expected: + return False + length, length_mask_expected = struct.unpack('' % (self.__class__.__name__, self.path, self.firstPos, self.lastPos) + PartialSplit(i + 1, first_edge + i * splitSize, first_edge + (i + 1) * splitSize) + for i in range(ns) + ] + [PartialSplit(ns + 1, last_edge, lastPos)] + self.repr_name = '<%s %s (%d-%d)>' % (self.__class__.__name__, path, firstPos, lastPos) class GZipFileRDD(TextFileRDD): @@ -1168,111 +1815,126 @@ def __init__(self, ctx, path, splitSize=None): def find_block(self, f, pos): f.seek(pos) - block = f.read(32*1024) + block = f.read(32 * 1024) if len(block) < 4: f.seek(0, 2) - return f.tell() # EOF - ENDING = '\x00\x00\xff\xff' + return f.tell() # EOF + ENDING = b'\x00\x00\xff\xff' while True: p = block.find(ENDING) while p < 0: pos += max(len(block) - 3, 0) - block = block[-3:] + f.read(32<<10) + block = block[-3:] + f.read(32 << 10) if len(block) < 4: - return pos + 3 # EOF + return pos + 3 # EOF p = block.find(ENDING) pos += p + 4 - block = block[p+4:] + block = block[p + 4:] if len(block) < 4096: block += f.read(4096) if not block: - return pos # EOF + return pos # EOF try: dz = zlib.decompressobj(-zlib.MAX_WBITS) if dz.decompress(block) and len(dz.unused_data) <= 8: - return pos # FOUND - except Exception, e: + return pos # FOUND + except Exception as e: pass def compute(self, split): - f = self.open_file() - last_line = '' - if split.index == 0: - zf = gzip.GzipFile(fileobj=f) - zf._read_gzip_header() - start = f.tell() - else: - start = self.find_block(f, split.index * self.splitSize) - if start >= split.index * self.splitSize + self.splitSize: - return - for i in xrange(1, 100): - if start - i * self.BLOCK_SIZE <= 4: - break - last_block = self.find_block(f, start - i * self.BLOCK_SIZE) - if last_block < start: - f.seek(last_block) - d = f.read(start - last_block) - dz = zlib.decompressobj(-zlib.MAX_WBITS) - last_line = dz.decompress(d).split('\n')[-1] - if last_line.endswith('\n'): - last_line = '' - break - - end = self.find_block(f, split.index * self.splitSize + self.splitSize) - # TODO: speed up - f.seek(start) - if self.fileinfo: - f.length = end - dz = zlib.decompressobj(-zlib.MAX_WBITS) - skip_first = False - while start < end: - d = f.read(min(64<<10, end-start)) - start += len(d) - if not d: break - - try: - io = StringIO(dz.decompress(d)) - except Exception, e: - if self.err < 1e-6: - logger.error("failed to decompress file: %s", self.path) - raise - old = start - start = self.find_block(f, start) - f.seek(start) - logger.error("drop corrupted block (%d bytes) in %s", - start - old + len(d), self.path) - skip_first = True - continue - - if len(dz.unused_data) > 8 : - f.seek(-len(dz.unused_data)+8, 1) - zf = gzip.GzipFile(fileobj=f) - zf._read_gzip_header() - dz = zlib.decompressobj(-zlib.MAX_WBITS) - start -= f.tell() - - last_line += io.readline() - if skip_first: - skip_first = False - elif last_line.endswith('\n'): - yield last_line[:-1] - last_line = '' + with closing(self.open_file()) as f: + last_line = b'' + if split.index == 0: + zf = gzip.GzipFile(mode='r', fileobj=f) + if hasattr(zf, '_buffer'): + zf._buffer.raw._read_gzip_header() + else: + zf._read_gzip_header() - ll = list(io) - if not ll: continue + zf.close() - last_line = ll.pop() - for line in ll: - yield line[:-1] - if last_line.endswith('\n'): - yield last_line[:-1] - last_line = '' + start = f.tell() + else: + start = self.find_block(f, split.index * self.splitSize) + if start >= split.index * self.splitSize + self.splitSize: + return + for i in range(1, 100): + if start - i * self.BLOCK_SIZE <= 4: + break + last_block = self.find_block(f, start - i * self.BLOCK_SIZE) + if last_block < start: + f.seek(last_block) + d = f.read(start - last_block) + dz = zlib.decompressobj(-zlib.MAX_WBITS) + _, sep, last_line = dz.decompress(d).rpartition(b'\n') + if sep: + break + + end = self.find_block(f, split.index * self.splitSize + self.splitSize) + # TODO: speed up + f.seek(start) + f.length = end + dz = zlib.decompressobj(-zlib.MAX_WBITS) + skip_first = False + while start < end: + d = f.read(min(64 << 10, end - start)) + start += len(d) + if not d: + break - f.close() + try: + io = BytesIO(dz.decompress(d)) + except Exception: + if self.err_ratio < 1e-6: + logger.error("failed to decompress file: %s", self.path) + raise + old = start + start = self.find_block(f, start) + f.seek(start) + logger.error("drop corrupted block (%d bytes) in %s", + start - old + len(d), self.path) + skip_first = True + continue + + if len(dz.unused_data) > 8: + f.seek(-len(dz.unused_data) + 8, 1) + zf = gzip.GzipFile(mode='r', fileobj=f) + if hasattr(zf, '_buffer'): + zf._buffer.raw._read_gzip_header() + else: + zf._read_gzip_header() + zf.close() + dz = zlib.decompressobj(-zlib.MAX_WBITS) + start -= f.tell() + + last_line += io.readline() + if skip_first: + skip_first = False + elif last_line.endswith(b'\n'): + line = last_line[:-1] + if not six.PY2: + line = line.decode('utf-8') + yield line + last_line = b'' + + ll = list(io) + if not ll: continue + + last_line = ll.pop() + for line in ll: + line = line[:-1] + if not six.PY2: + line = line.decode('utf-8') + yield line + if last_line.endswith(b'\n'): + line = last_line[:-1] + if not six.PY2: + line = line.decode('utf-8') + yield line + last_line = b'' class TableFileRDD(TextFileRDD): - DEFAULT_SPLIT_SIZE = 32 << 20 def __init__(self, ctx, path, splitSize=None): @@ -1280,13 +1942,13 @@ def __init__(self, ctx, path, splitSize=None): def find_magic(self, f, pos, magic): f.seek(pos) - block = f.read(32*1024) + block = f.read(32 * 1024) if len(block) < len(magic): return -1 p = block.find(magic) while p < 0: pos += len(block) - len(magic) + 1 - block = block[1 - len(magic):] + f.read(32<<10) + block = block[1 - len(magic):] + f.read(32 << 10) if len(block) == len(magic) - 1: return -1 p = block.find(magic) @@ -1294,83 +1956,109 @@ def find_magic(self, f, pos, magic): def compute(self, split): import msgpack - f = self.open_file() - magic = f.read(8) - start = split.index * self.splitSize - end = (split.index + 1) * self.splitSize - start = self.find_magic(f, start, magic) - if start < 0: - return - f.seek(start) - hdr_size = 12 - while start < end: - m = f.read(len(magic)) - if m != magic: - break - compressed, count, size = struct.unpack("III", f.read(hdr_size)) - d = f.read(size) - assert len(d) == size, 'unexpected end' - if compressed: - d = zlib.decompress(d) - for r in msgpack.Unpacker(StringIO(d)): - yield r - start += len(magic) + hdr_size + size - f.close() + with closing(self.open_file()) as f: + magic = f.read(8) + start = split.index * self.splitSize + end = (split.index + 1) * self.splitSize + start = self.find_magic(f, start, magic) + if start < 0: + return + f.seek(start) + hdr_size = 12 + while start < end: + m = f.read(len(magic)) + if m != magic: + break + compressed, count, size = struct.unpack("III", f.read(hdr_size)) + d = f.read(size) + assert len(d) == size, 'unexpected end' + if compressed: + d = zlib.decompress(d) + for r in msgpack.Unpacker(BytesIO(d)): + yield r + start += len(magic) + hdr_size + size class BZip2FileRDD(TextFileRDD): "the bzip2ed file must be seekable, compressed by pbzip2" - DEFAULT_SPLIT_SIZE = 32*1024*1024 + DEFAULT_SPLIT_SIZE = 32 * 1024 * 1024 BLOCK_SIZE = 9000 def __init__(self, ctx, path, numSplits=None, splitSize=None): TextFileRDD.__init__(self, ctx, path, numSplits, splitSize) def compute(self, split): - f = self.open_file() - magic = f.read(10) - f.seek(split.index * self.splitSize) - d = f.read(self.splitSize) - fp = d.find(magic) - if fp > 0: - d = d[fp:] # drop end of last block - - # real all the block - nd = f.read(self.BLOCK_SIZE) - np = nd.find(magic) - while nd and np < 0: - t = f.read(len(nd)) - if not t: break - nd += t + with closing(self.open_file()) as f: + magic = f.read(10) + f.seek(split.index * self.splitSize) + d = f.read(self.splitSize) + fp = d.find(magic) + if fp > 0: + d = d[fp:] # drop end of last block + + # real all the block + nd = f.read(self.BLOCK_SIZE) np = nd.find(magic) - d += nd[:np] if np >= 0 else nd - f.close() + while nd and np < 0: + t = f.read(len(nd)) + if not t: + break + nd += t + np = nd.find(magic) + d += nd[:np] if np >= 0 else nd + + last_line = b'' + if split.index > 0: + cur = split.index * self.splitSize + skip = fp if fp >= 0 else d.find(magic) + if skip >= 0: + cur += skip + else: + cur += len(d) + + for i in range(1, 100): + pos = cur - i * self.BLOCK_SIZE + if pos < 0: + break + + f.seek(pos) + nd = f.read(cur - pos) + np = nd.find(magic) + if np >= 0: + nd = nd[np:] + last_line = bz2.decompress(nd).split(b'\n')[-1] + break - last_line = None if split.index > 0 else '' while d: + np = d.find(magic, len(magic)) + if np <= 0: + data = d + else: + data = d[:np] try: - io = StringIO(bz2.decompress(d)) - except IOError, e: - #bad position, skip it + io = BytesIO(bz2.decompress(data)) + except IOError as e: + # bad position, skip it pass else: - if last_line is None: - io.readline() # skip the first line - last_line = '' - else: - last_line += io.readline() - if last_line.endswith('\n'): - yield last_line[:-1] - last_line = '' + last_line += io.readline() + if last_line.endswith(b'\n'): + line = last_line[:-1] + if not six.PY2: + line = line.decode('utf-8') + yield line + last_line = b'' for line in io: - if line.endswith('\n'): # drop last line - yield line[:-1] + if line.endswith(b'\n'): # drop last line + line = line[:-1] + if not six.PY2: + line = line.decode('utf-8') + yield line else: last_line = line - np = d.find(magic, len(magic)) if np <= 0: break d = d[np:] @@ -1378,36 +2066,32 @@ def compute(self, split): class BinaryFileRDD(TextFileRDD): def __init__(self, ctx, path, fmt=None, length=None, numSplits=None, splitSize=None): - TextFileRDD.__init__(self, ctx, path, numSplits, splitSize) self.fmt = fmt if fmt: length = struct.calcsize(fmt) self.length = length assert length, "fmt or length must been provided" + if splitSize is None: + splitSize = self.DEFAULT_SPLIT_SIZE - self.splitSize = max(self.splitSize / length, 1) * length - n = self.size / self.splitSize - if self.size % self.splitSize > 0: - n += 1 - self.len = n - - def __repr__(self): - return '' % (self.fmt, self.path) + splitSize = max(splitSize // length, 1) * length + TextFileRDD.__init__(self, ctx, path, numSplits, splitSize) + self.repr_name = '' % (fmt, path) def compute(self, split): start = split.index * self.splitSize end = min(start + self.splitSize, self.size) - f = self.open_file() - f.seek(start) - rlen = self.length - fmt = self.fmt - for i in xrange((end - start) / rlen): - d = f.read(rlen) - if len(d) < rlen: break - if fmt: - d = struct.unpack(fmt, d) - yield d + with closing(self.open_file()) as f: + f.seek(start) + rlen = self.length + fmt = self.fmt + for i in range((end - start) // rlen): + d = f.read(rlen) + if len(d) < rlen: break + if fmt: + d = struct.unpack(fmt, d) + yield d class OutputTextFileRDD(DerivedRDD): @@ -1434,42 +2118,33 @@ def __init__(self, rdd, path, ext='', overwrite=False, compress=False): self.ext = ext self.overwrite = overwrite self.compress = compress - - def __repr__(self): - return '<%s %s %s>' % (self.__class__.__name__, self.path, self.prev) + self.repr_name = '<%s %s %s>' % (self.__class__.__name__, path, rdd) def compute(self, split): path = os.path.join(self.path, - "%04d%s" % (split.index, self.ext)) + "%04d%s" % (split.index, self.ext)) if os.path.exists(path) and not self.overwrite: return - tpath = os.path.join(self.path, - ".%04d%s.%s.%d.tmp" % (split.index, self.ext, - socket.gethostname(), os.getpid())) - try: - try: - f = open(tpath,'w', 4096 * 1024 * 16) - except IOError: - time.sleep(1) # there are dir cache in mfs for 1 sec - f = open(tpath,'w', 4096 * 1024 * 16) + + with atomic_file(path, mode='wb', bufsize=4096 * 1024 * 16) as f: if self.compress: have_data = self.write_compress_data(f, self.prev.iterator(split)) else: have_data = self.writedata(f, self.prev.iterator(split)) - f.close() - if have_data and not os.path.exists(path): - os.rename(tpath, path) - yield path - finally: - try: - os.remove(tpath) - except: - pass + + if not have_data: + raise AbortFileReplacement + + if os.path.exists(path): + yield path def writedata(self, f, lines): + if not six.PY2: + f = TextIOWrapper(f) + it = iter(lines) try: - line = it.next() + line = next(it) except StopIteration: return False f.write(line) @@ -1481,27 +2156,50 @@ def writedata(self, f, lines): if s: f.write(s) f.write('\n') + if not six.PY2: + f.close() return True def write_compress_data(self, f, lines): empty = True - f = gzip.GzipFile(filename='', mode='w', fileobj=f) - size = 0 - for line in lines: - f.write(line) - if not line.endswith('\n'): - f.write('\n') - size += len(line) + 1 - if size >= 256 << 10: + with gzip.GzipFile(filename='', mode='w', fileobj=f) as f: + if not six.PY2: + f = TextIOWrapper(f) + size = 0 + for line in lines: + f.write(line) + if not line.endswith('\n'): + f.write('\n') + size += len(line) + 1 + if size >= 256 << 10: + f.flush() + f.compress = zlib.compressobj(9, zlib.DEFLATED, + -zlib.MAX_WBITS, zlib.DEF_MEM_LEVEL, 0) + size = 0 + empty = False + if not empty: f.flush() - f.compress = zlib.compressobj(9, zlib.DEFLATED, - -zlib.MAX_WBITS, zlib.DEF_MEM_LEVEL, 0) - size = 0 + if not six.PY2: + f.close() + + return not empty + + +class OutputTfrecordstFileRDD(OutputTextFileRDD): + def __init__(self, rdd, path, ext, overwrite=True, compress=False): + OutputTextFileRDD.__init__(self, rdd=rdd, path=path, ext='.tfrecords', overwrite=overwrite, compress=compress) + + def writedata(self, f, strings): + empty = True + for string in strings: + string_bytes = str(string).encode() + encoded_length = struct.pack(' self.MAX_OPEN_FILES: if self.compress: @@ -1586,7 +2291,7 @@ def compute(self, split): for k, f in self.files.items(): if self.compress: - f = self.get_file(k) # make sure fileobj is open + f = self.get_file(k) # make sure fileobj is open f.close() for k, tpath in self.paths.items(): @@ -1608,6 +2313,9 @@ def __init__(self, rdd, path, dialect, overwrite, compress): self.dialect = dialect def writedata(self, f, rows): + if not six.PY2: + f = TextIOWrapper(f) + writer = csv.writer(f, self.dialect) empty = True for row in rows: @@ -1615,27 +2323,34 @@ def writedata(self, f, rows): row = (row,) writer.writerow(row) empty = False + if not six.PY2: + f.close() return not empty def write_compress_data(self, f, rows): empty = True - f = gzip.GzipFile(filename='', mode='w', fileobj=f) - writer = csv.writer(f, self.dialect) - last_flush = 0 - for row in rows: - if not isinstance(row, (tuple, list)): - row = (row,) - writer.writerow(row) - empty = False - if f.tell() - last_flush >= 256 << 10: + with gzip.GzipFile(filename='', mode='w', fileobj=f) as f: + if not six.PY2: + f = TextIOWrapper(f) + writer = csv.writer(f, self.dialect) + last_flush = 0 + for row in rows: + if not isinstance(row, (tuple, list)): + row = (row,) + writer.writerow(row) + empty = False + if f.tell() - last_flush >= 256 << 10: + f.flush() + f.compress = zlib.compressobj(9, zlib.DEFLATED, + -zlib.MAX_WBITS, zlib.DEF_MEM_LEVEL, 0) + last_flush = f.tell() + if not empty: f.flush() - f.compress = zlib.compressobj(9, zlib.DEFLATED, - -zlib.MAX_WBITS, zlib.DEF_MEM_LEVEL, 0) - last_flush = f.tell() - if not empty: - f.flush() + if not six.PY2: + f.close() return not empty + class OutputBinaryFileRDD(OutputTextFileRDD): def __init__(self, rdd, path, fmt, overwrite): OutputTextFileRDD.__init__(self, rdd, path, '.bin', overwrite) @@ -1651,9 +2366,10 @@ def writedata(self, f, rows): empty = False return not empty + class OutputTableFileRDD(OutputTextFileRDD): - MAGIC = '\x00\xDE\x00\xAD\xFF\xBE\xFF\xEF' - BLOCK_SIZE = 256 << 10 # 256K + MAGIC = b'\x00\xDE\x00\xAD\xFF\xBE\xFF\xEF' + BLOCK_SIZE = 256 << 10 # 256K def __init__(self, rdd, path, overwrite=True, compress=True): OutputTextFileRDD.__init__(self, rdd, path, ext='.tab', overwrite=overwrite, compress=False) @@ -1661,6 +2377,7 @@ def __init__(self, rdd, path, overwrite=True, compress=True): def writedata(self, f, rows): import msgpack + def flush(buf): d = buf.getvalue() if self.compress: @@ -1669,13 +2386,13 @@ def flush(buf): f.write(struct.pack("III", self.compress, count, len(d))) f.write(d) - count, buf = 0, StringIO() + count, buf = 0, BytesIO() for row in rows: msgpack.pack(row, buf) count += 1 if buf.tell() > self.BLOCK_SIZE: flush(buf) - count, buf = 0, StringIO() + count, buf = 0, BytesIO() if count > 0: flush(buf) @@ -1684,338 +2401,30 @@ def flush(buf): write_compress_data = writedata -# -# Beansdb -# -import marshal -import binascii -try: - import quicklz -except ImportError: - quicklz = None - -try: - from fnv1a import get_hash - from fnv1a import get_hash_beansdb - def fnv1a(d): - return get_hash(d) & 0xffffffff - def fnv1a_beansdb(d): - return get_hash_beansdb(d) & 0xffffffff -except ImportError: - FNV_32_PRIME = 0x01000193 - FNV_32_INIT = 0x811c9dc5 - def fnv1a(d): - h = FNV_32_INIT - for c in d: - h ^= ord(c) - h *= FNV_32_PRIME - h &= 0xffffffff - return h - fnv1a_beansdb = fnv1a - - - -FLAG_PICKLE = 0x00000001 -FLAG_INTEGER = 0x00000002 -FLAG_LONG = 0x00000004 -FLAG_BOOL = 0x00000008 -FLAG_COMPRESS1= 0x00000010 # by cmemcached -FLAG_MARSHAL = 0x00000020 -FLAG_COMPRESS = 0x00010000 # by beansdb - -PADDING = 256 - -def restore_value(flag, val): - if flag & FLAG_COMPRESS: - val = quicklz.decompress(val) - if flag & FLAG_COMPRESS1: - val = zlib.decompress(val) - - if flag & FLAG_BOOL: - val = bool(int(val)) - elif flag & FLAG_INTEGER: - val = int(val) - elif flag & FLAG_MARSHAL: - val = marshal.loads(val) - elif flag & FLAG_PICKLE: - val = cPickle.loads(val) - return val - -def prepare_value(val, compress): - flag = 0 - if isinstance(val, str): - pass - elif isinstance(val, (bool)): - flag = FLAG_BOOL - val = str(int(val)) - elif isinstance(val, (int, long)): - flag = FLAG_INTEGER - val = str(val) - elif type(val) is unicode: - flag = FLAG_MARSHAL - val = marshal.dumps(val, 2) - else: - try: - val = marshal.dumps(val, 2) - flag = FLAG_MARSHAL - except ValueError: - val = cPickle.dumps(val, -1) - flag = FLAG_PICKLE - - if compress and len(val) > 1024: - flag |= FLAG_COMPRESS - val = quicklz.compress(val) - - return flag, val - class BeansdbFileRDD(TextFileRDD): - def __init__(self, ctx, path, filter=None, fullscan=False, raw=False): - if not fullscan: - hint = path[:-5] + '.hint' - if not os.path.exists(hint) and not os.path.exists(hint + '.qlz'): - fullscan = True - if not filter: - fullscan = True - TextFileRDD.__init__(self, ctx, path, numSplits=None if fullscan else 1) - self.func = filter - self.fullscan = fullscan - self.raw = raw - - @cached - def __getstate__(self): - d = RDD.__getstate__(self) - del d['func'] - return d, dump_func(self.func) - def __setstate__(self, state): - self.__dict__, code = state - try: - self.func = load_func(code) - except Exception: - print 'load failed', self.__class__, code[:1024] - raise + def __init__(self, ctx, path, filter=None, fullscan=False, raw=False): + key_filter = filter + if key_filter is None: + fullscan = True + TextFileRDD.__init__( + self, ctx, path, numSplits=None if fullscan else 1) + self.reader = BeansdbReader(path, key_filter, fullscan, raw) def compute(self, split): - if self.fullscan: - return self.full_scan(split) - hint = self.path[:-5] + '.hint.qlz' - if os.path.exists(hint): - return self.scan_hint(hint) - hint = self.path[:-5] + '.hint' - if os.path.exists(hint): - return self.scan_hint(hint) - return self.full_scan(split) - - def scan_hint(self, hint_path): - hint = open(hint_path).read() - if hint_path.endswith('.qlz'): - try: - hint = quicklz.decompress(hint) - except ValueError, e: - msg = str(e.message) - if msg.startswith('compressed length not match'): - hint = hint[:int(msg.split('!=')[1])] - hint = quicklz.decompress(hint) - - func = self.func or (lambda x:True) - dataf = open(self.path) - p = 0 - while p < len(hint): - pos, ver, hash = struct.unpack("IiH", hint[p:p+10]) - p += 10 - ksz = pos & 0xff - key = hint[p: p+ksz] - if func(key): - dataf.seek(pos & 0xffffff00) - r = self.read_record(dataf) - if r: - rsize, key, value = r - yield key, value - else: - logger.error("read failed from %s at %d", self.path, pos & 0xffffff00) - p += ksz + 1 # \x00 - - def restore(self, flag, val): - if self.raw: - return (flag, val) - return restore_value(flag, val) - - def try_read_record(self, f): - block = f.read(PADDING) - if not block: - return - - crc, tstamp, flag, ver, ksz, vsz = struct.unpack("IiiiII", block[:24]) - if not (0 < ksz < 255 and 0 <= vsz < (50<<20)): - return - rsize = 24 + ksz + vsz - if rsize & 0xff: - rsize = ((rsize >> 8) + 1) << 8 - if rsize > PADDING: - block += f.read(rsize-PADDING) - crc32 = binascii.crc32(block[4:24 + ksz + vsz]) & 0xffffffff - if crc != crc32: - return - return True - - def read_record(self, f): - block = f.read(PADDING) - if len(block) < 24: - return - - crc, tstamp, flag, ver, ksz, vsz = struct.unpack("IiiiII", block[:24]) - if not (0 < ksz < 255 and 0 <= vsz < (50<<20)): - logger.warning('bad key length %d %d', ksz, vsz) - return - - rsize = 24 + ksz + vsz - if rsize & 0xff: - rsize = ((rsize >> 8) + 1) << 8 - if rsize > PADDING: - block += f.read(rsize-PADDING) - #crc32 = binascii.crc32(block[4:24 + ksz + vsz]) & 0xffffffff - #if crc != crc32: - # print 'crc broken', crc, crc32 - # return - key = block[24:24+ksz] - value = block[24+ksz:24+ksz+vsz] - if not self.func or self.func(key): - value = self.restore(flag, value) - return rsize, key, (value, ver, tstamp) - - def full_scan(self, split): - f = self.open_file() - - # try to find first record - begin, end = split.begin, split.end - while True: - f.seek(begin) - r = self.try_read_record(f) - if r: break - begin += PADDING - if begin >= end: break - if begin >= end: - return - - f.seek(begin) - func = self.func or (lambda x:True) - while begin < end: - r = self.read_record(f) - if not r: - begin += PADDING - logger.error('read fail at %s pos: %d', self.path, begin) - while begin < end: - f.seek(begin) - if self.try_read_record(f): - break - begin += PADDING - continue - size, key, value = r - if func(key): - yield key, value - begin += size + return self.reader.read(split.begin, split.end) class OutputBeansdbRDD(DerivedRDD): - def __init__(self, rdd, path, depth, overwrite, compress=False, raw=False): - DerivedRDD.__init__(self, rdd) - self.path = path - self.depth = depth - self.overwrite = overwrite - if not quicklz: - compress = False - self.compress = compress - self.raw = raw - - for i in range(16 ** depth): - if depth > 0: - ps = list(('%%0%dx' % depth) % i) - p = os.path.join(path, *ps) - else: - p = path - if os.path.exists(p): - if overwrite: - for n in os.listdir(p): - if n[:3].isdigit(): - os.remove(os.path.join(p, n)) - else: - os.makedirs(p) - - - def __repr__(self): - return '<%s %s %s>' % (self.__class__.__name__, self.path, self.prev) - - def prepare(self, val): - if self.raw: - return val - - return prepare_value(val, self.compress) - def gen_hash(self, d): - # used in beansdb - h = len(d) * 97 - if len(d) <= 1024: - h += fnv1a_beansdb(d) - else: - h += fnv1a_beansdb(d[:512]) - h *= 97 - h += fnv1a_beansdb(d[-512:]) - return h & 0xffff - - def write_record(self, f, key, flag, value, now=None): - header = struct.pack('IIIII', now, flag, 1, len(key), len(value)) - crc32 = binascii.crc32(header) - crc32 = binascii.crc32(key, crc32) - crc32 = binascii.crc32(value, crc32) & 0xffffffff - f.write(struct.pack("I", crc32)) - f.write(header) - f.write(key) - f.write(value) - rsize = 24 + len(key) + len(value) - if rsize & 0xff: - f.write('\x00' * (PADDING - (rsize & 0xff))) - rsize = ((rsize >> 8) + 1) << 8 - return rsize + def __init__(self, rdd, path, depth, overwrite, compress=False, + raw=False, value_with_meta=False): + DerivedRDD.__init__(self, rdd) + self.writer = BeansdbWriter(path, depth, overwrite, compress, + raw, value_with_meta) + self.repr_name = '<%s %s %s>' % ( + self.__class__.__name__, path, self.prev) def compute(self, split): - N = 16 ** self.depth - if self.depth > 0: - fmt='%%0%dx' % self.depth - ds = [os.path.join(self.path, *list(fmt % i)) for i in range(N)] - else: - ds = [self.path] - pname = '%03d.data' % split.index - tname = '.%03d.data.%s.tmp' % (split.index, socket.gethostname()) - p = [os.path.join(d, pname) for d in ds] - tp = [os.path.join(d, tname) for d in ds] - pos = [0] * N - f = [open(t, 'w', 1<<20) for t in tp] - now = int(time.time()) - hint = [[] for d in ds] - - bits = 32 - self.depth * 4 - for key, value in self.prev.iterator(split): - key = str(key) - i = fnv1a(key) >> bits - flag, value = self.prepare(value) - h = self.gen_hash(value) - hint[i].append(struct.pack("IIH", pos[i] + len(key), 1, h) + key + '\x00') - pos[i] += self.write_record(f[i], key, flag, value, now) - if pos[i] > (4000<<20): - raise Exception("split is large than 4000M") - [i.close() for i in f] - - for i in range(N): - if hint[i] and not os.path.exists(p[i]): - os.rename(tp[i], p[i]) - hintdata = ''.join(hint[i]) - hint_path = os.path.join(os.path.dirname(p[i]), '%03d.hint' % split.index) - if self.compress: - hintdata = quicklz.compress(hintdata) - hint_path += '.qlz' - open(hint_path, 'w').write(hintdata) - else: - os.remove(tp[i]) - - return sum([([p[i]] if hint[i] else []) for i in range(N)], []) + return self.writer.write_bucket(self.prev.iterator(split), split.index) diff --git a/dpark/schedule.py b/dpark/schedule.py index c3a46da5..75bea8af 100644 --- a/dpark/schedule.py +++ b/dpark/schedule.py @@ -1,83 +1,118 @@ -import os, sys -import socket -import logging +from __future__ import absolute_import import marshal -import cPickle -import threading, Queue +import multiprocessing +import os +import socket +import sys import time -import random -import urllib -import warnings +import six +from six.moves import map, range, urllib, queue, cPickle import weakref -import multiprocessing -import platform +import threading +import json +from collections import Counter import zmq +from addict import Dict +from pymesos import MesosSchedulerDriver, encode_data, decode_data -import pymesos as mesos -import pymesos.mesos_pb2 as mesos_pb2 - -from dpark.util import compress, decompress, spawn, getuser -from dpark.dependency import NarrowDependency, ShuffleDependency +import dpark.conf as conf from dpark.accumulator import Accumulator -from dpark.task import ResultTask, ShuffleMapTask -from dpark.job import SimpleJob +from dpark.dependency import ShuffleDependency from dpark.env import env +from dpark.taskset import TaskSet, TaskCounter from dpark.mutable_dict import MutableDict +from dpark.task import ResultTask, ShuffleMapTask, TTID, TaskState, TaskEndReason +from dpark.hostatus import TaskHostManager +from dpark.shuffle import MapOutputTracker +from dpark.utils import ( + compress, decompress, spawn, getuser, + sec2nanosec) +from dpark.utils.log import get_logger +from dpark.utils.frame import Scope +from dpark.utils import dag + -logger = logging.getLogger(__name__) +logger = get_logger(__name__) -MAX_FAILED = 3 -EXECUTOR_MEMORY = 64 # cache +EXECUTOR_CPUS = 0.01 +EXECUTOR_MEMORY = 128 # cache POLL_TIMEOUT = 0.1 RESUBMIT_TIMEOUT = 60 MAX_IDLE_TIME = 60 * 30 -class TaskEndReason: pass -class Success(TaskEndReason): pass -class FetchFailed(TaskEndReason): - def __init__(self, serverUri, shuffleId, mapId, reduceId): - self.serverUri = serverUri - self.shuffleId = shuffleId - self.mapId = mapId - self.reduceId = reduceId - def __str__(self): - return '' % (self.serverUri, - self.shuffleId, self.mapId, self.reduceId) -class OtherFailure(TaskEndReason): - def __init__(self, message): - self.message = message - def __str__(self): - return '' % self.message +class Stage(object): + + def __init__(self, rdd, shuffleDep, parents, pipelines, pipeline_edges, rdd_pipelines): + """ -class Stage: - def __init__(self, rdd, shuffleDep, parents): - self.id = self.newId() + :param rdd: output rdd of this stage + :param shuffleDep: for mapOutputStage, determine how computing result will be aggregated, partitioned + """ + self.id = self.new_id() + self.num_try = 0 self.rdd = rdd self.shuffleDep = shuffleDep + self.is_final = (shuffleDep is None) self.parents = parents self.numPartitions = len(rdd) - self.outputLocs = [[] for i in range(self.numPartitions)] + self.num_finished = 0 # for final stage + self.outputLocs = [[] for _ in range(self.numPartitions)] + self.task_stats = [[] for _ in range(self.numPartitions)] + self.taskcounters = [] # a TaskCounter object for each run/retry + self.submit_time = 0 + self.finish_time = 0 + self.pipelines = pipelines # each pipeline is a list of rdds + self.pipeline_edges = pipeline_edges # ((src_stage_id, src_pipeline_id), (dst_stage_id, dst_pipeline_id)): N + self.rdd_pipelines = rdd_pipelines # rdd_id: pipeline_id def __str__(self): return '' % (self.id, self.rdd) def __getstate__(self): - raise Exception("should not pickle stage") + raise Exception('should not pickle stage') + + def __len__(self): + return self.numPartitions + + nextId = 0 + + @classmethod + def new_id(cls): + cls.nextId += 1 + return cls.nextId + + @property + def try_id(self): + return TTID.make_taskset_id(self.id, self.num_try + 1) # incr num_try After create TaskSet @property def isAvailable(self): - if not self.parents and self.shuffleDep == None: + if not self.parents and self.shuffleDep is None: return True return all(self.outputLocs) + @property + def num_task_finished(self): + if self.is_final: + return self.num_finished + else: + return len([i for i in self.outputLocs if i]) + + @property + def num_task_running(self): + if self.taskcounters: + return self.taskcounters[-1].running + else: + return 0 + def addOutputLoc(self, partition, host): self.outputLocs[partition].append(host) -# def removeOutput(self, partition, host): -# prev = self.outputLocs[partition] -# self.outputLocs[partition] = [h for h in prev if h != host] + # def removeOutput(self, partition, host): + # prev = self.outputLocs[partition] + # self.outputLocs[partition] = [h for h in prev if h != host] def removeHost(self, host): becameUnavailable = False @@ -86,95 +121,328 @@ def removeHost(self, host): ls.remove(host) becameUnavailable = True if becameUnavailable: - logger.info("%s is now unavailable on host %s", self, host) - - nextId = 0 - @classmethod - def newId(cls): - cls.nextId += 1 - return cls.nextId + msg = ("%s is now unavailable on host %s, " + "postpone resubmit until %d secs later " + "to wait for futher fetch failure") + logger.info(msg, self, host, RESUBMIT_TIMEOUT) + + def finish(self): + if not self.finish_time: + self.finish_time = time.time() + + def _summary_stats(self): + stats = [x[-1] for x in self.task_stats if x] + + d = {} + + def _summary(lst): + lst.sort() + r = {'max': max(lst), + 'min': min(lst), + 'sum': sum(lst) + } + return r + + if stats: + for attr in dir(stats[0]): + if not attr.startswith('_'): + d[attr] = _summary(list([getattr(s, attr) for s in stats])) + return d + + def get_node_id(self, stage_id, pipeline_id): + if stage_id == -1: + stage_id = self.id + return "PIPELINE_{}.{}".format(stage_id, pipeline_id) + + def _fmt_node(self, stage_id, pipeline_id): + if stage_id == -1: + stage_id = self.id + rdds = [{"rdd_name": rdd.ui_label, + "rdd_id": rdd.id, + "api_callsite_id": rdd.scope.api_callsite_id, + "params": rdd.params} + for rdd in self.pipelines[pipeline_id]] + n = { + dag.KW_TYPE: "stage", + dag.KW_ID: self.get_node_id(stage_id, pipeline_id), + dag.KW_LABEL: str(stage_id), + "rdds": rdds + } + return n + + def _fmt_edge(self, e): + e, nrdd = e + src, dst = [self.get_node_id(*n) for n in e] + info = {} + if nrdd > 1: + info['#rdd'] = nrdd + + return { + # dag.KW_ID: "{}_{}".format(src, dst), + dag.KW_SRC: src, + dag.KW_DST: dst, + "info": info + } + + def get_pipeline_graph(self): + nodes = [self._fmt_node(self.id, pipeline_id) for pipeline_id in self.pipelines.keys()] + edges = [self._fmt_edge(e) for e in six.iteritems(self.pipeline_edges)] + g = {dag.KW_NODES: nodes, dag.KW_EDGES: edges} + return g + + def fmt_stats(self): + n = self.numPartitions + stats = self._summary_stats() + msg = "[metric = min/avg/max]: " + for k, d in six.iteritems(stats): + sm = d['sum'] + unit = k.split('_')[0] + if unit == 'num': + fmt = " = %d/%d/%d" + else: + fmt = " = %.2f/%.2f/%.2f" + + if sm > 0: + msg += k + vs = d['min'], sm / n, d['max'] + vs = tuple(map(int, vs)) + unit_s = " " + if unit == "bytes": + vs = tuple([v >> 20 for v in vs]) + fmt = " = %.2f/%.2f/%.2f" + unit_s = " MB " + msg += (fmt % vs) + msg += unit_s + return msg + + def _summary_counters(self): + + def _sum(attr): + return sum([getattr(counter, attr) for counter in self.taskcounters]) + + counters = { + "task": { + "all": len(self), + "running": self.num_task_running, + "finished": self.num_task_finished, + }, + "fail": dict([(attr[5:], _sum(attr)) for attr in TaskCounter(0).get_fail_types()]) + } + return counters + + def get_prof(self): + stats = self._summary_stats() + counters = self._summary_counters() + graph = self.get_pipeline_graph() + + info = { + 'id': self.id, + 'parents': [p.id for p in self.parents], + 'output_rdd': self.rdd.__class__.__name__, + 'output_pipeline': self.get_node_id(self.id, self.rdd.id), + 'api_callsite': self.rdd.scope.api_callsite, + 'start_time': self.submit_time, + 'finish_time': self.finish_time, + 'num_partition': self.numPartitions, + 'mem': self.rdd.mem, + } + res = { + "info": info, + "stats": stats, + "counters": counters, + 'graph': graph + } + return res + + def get_tree_stages(self): + stages = [] + to_visit = [self] + seen = set() + while to_visit: + s = to_visit.pop() + stages.append(s) + for ss in s.parents: + if ss.id not in seen: + to_visit.append(ss) + seen.add(ss.id) + + return stages class Scheduler: - def start(self): pass - def runJob(self, rdd, func, partitions, allowLocal): pass - def clear(self): pass - def stop(self): pass + + def start(self): + pass + + def runJob(self, rdd, func, partitions, allowLocal): + pass + + def clear(self): + pass + + def stop(self): + pass + def defaultParallelism(self): return 2 + class CompletionEvent: - def __init__(self, task, reason, result, accumUpdates): + + def __init__(self, task, reason, result, accumUpdates, stats): self.task = task self.reason = reason self.result = result self.accumUpdates = accumUpdates + self.stats = stats + + +def walk_dependencies(rdd, edge_func=lambda r, d: True, node_func=lambda r: True): + visited = set() + to_visit = [rdd] + while to_visit: + r = to_visit.pop(0) + if r.id in visited: + continue + visited.add(r.id) + if node_func(r): + for dep in r.dependencies: + if edge_func(r, dep): + to_visit.append(dep.rdd) class DAGScheduler(Scheduler): def __init__(self): - self.completionEvents = Queue.Queue() + self.id = self.new_id() + self.completionEvents = queue.Queue() self.idToStage = weakref.WeakValueDictionary() self.shuffleToMapStage = {} self.cacheLocs = {} - self._shutdown = False + self.idToRunJob = {} + self.runJobTimes = 0 + self.frameworkId = None + self.loghub_dir = None + self.jobstats = [] + self.is_dstream = False + self.current_scope = None + + self.final_lock = threading.RLock() + self.final_stage = None + self.final_rdd = None - def check(self): - pass + nextId = 0 + + @classmethod + def new_id(cls): + cls.nextId += 1 + return cls.nextId def clear(self): self.idToStage.clear() self.shuffleToMapStage.clear() self.cacheLocs.clear() - self.cacheTracker.clear() - self.mapOutputTracker.clear() - - def shutdown(self): - self._shutdown = True - - @property - def cacheTracker(self): - return env.cacheTracker - - @property - def mapOutputTracker(self): - return env.mapOutputTracker def submitTasks(self, tasks): raise NotImplementedError - def taskEnded(self, task, reason, result, accumUpdates): - self.completionEvents.put(CompletionEvent(task, reason, result, accumUpdates)) + def taskEnded(self, task, reason, result, accumUpdates, stats=None): + self.completionEvents.put( + CompletionEvent( + task, + reason, + result, + accumUpdates, + stats)) + + def abort(self): + self.completionEvents.put(None) def getCacheLocs(self, rdd): - return self.cacheLocs.get(rdd.id, [[] for i in range(len(rdd))]) + return self.cacheLocs.get(rdd.id, [[] for _ in range(len(rdd))]) def updateCacheLocs(self): - self.cacheLocs = self.cacheTracker.getLocationsSnapshot() + self.cacheLocs = env.cacheTrackerServer.getLocationsSnapshot() - def newStage(self, rdd, shuffleDep): - stage = Stage(rdd, shuffleDep, self.getParentStages(rdd)) - self.idToStage[stage.id] = stage - logger.debug("new stage: %s", stage) - return stage + def newStage(self, output_rdd, shuffleDep): + """ A stage may contain multi data pipeline, which form a tree with one final output pipline as root. + Zip, CartesianRDD, and Union may commine diff data sources, so lead to a split of the tree. + The leaves of the tree may be one of: + 1. a pipeline start from a source RDD (TextFileRDD, Collection) + 2. a root pipeline of a parent stage . + Unioned rdds with same lineage keep only one by add it to dep_rdds and assign a pipeline_id. + + --- + + Be careful: + - On one hand, logic for ui should not risk mixing newStage, the latter is much more important. + - On the other hand, input pipeline need to link to parent stages. + """ + parent_stages = set() + + pipelines = {output_rdd.id: [output_rdd]} + pipeline_edges = Counter() + + rdd_pipelines = {output_rdd.id: output_rdd.id} - def getParentStages(self, rdd): - parents = set() + to_visit = [output_rdd] visited = set() - def visit(r): + dep_filter = set() + + while to_visit: + r = to_visit.pop(0) if r.id in visited: - return + continue visited.add(r.id) + my_pipeline_id = rdd_pipelines.get(r.id) + if my_pipeline_id is not None: # not all rdd have my_pipeline_id + my_pipeline = pipelines.get(my_pipeline_id) + if my_pipeline is None: + logger.warning("miss pipeline: {} ".format(r.scope.key)) + if r.shouldCache: - self.cacheTracker.registerRDD(r.id, len(r)) + env.cacheTrackerServer.registerRDD(r.id, len(r)) + + dep_rdds = [] + dep_stages = [] for dep in r.dependencies: if isinstance(dep, ShuffleDependency): - parents.add(self.getShuffleMapStage(dep)) + stage = self.getShuffleMapStage(dep) + parent_stages.add(stage) + dep_stages.append(stage) + if my_pipeline_id is not None: + pipeline_edges[(stage.id, stage.rdd.id), (-1, my_pipeline_id)] += 1 # -1 : current_stage + else: + logger.warning("miss pipeline: {} {}".format(r.scope.key, dep.rdd.scope.key)) else: - visit(dep.rdd) - visit(rdd) - return list(parents) + to_visit.append(dep.rdd) + if r.id not in dep_filter and dep.rdd.id in r.dep_lineage_counts: + dep_rdds.append(dep.rdd) + else: + dep_filter.add(dep.rdd.id) + + if my_pipeline is None: + continue + + ns, nr = len(dep_stages), len(dep_rdds) + + if ns + nr <= 1: + if nr == 1: + dep_rdd = dep_rdds[0] + my_pipeline.append(dep_rdd) + rdd_pipelines[dep_rdd.id] = my_pipeline_id + else: + for dep_rdd in dep_rdds: + did = dep_rdd.id + nrdd = r.dep_lineage_counts[did] + pipelines[did] = [dep_rdd] # create a new pipeline/branch + rdd_pipelines[did] = did + pipeline_edges[((-1, did), (-1, my_pipeline_id))] = nrdd # -1 : current_stage + + stage = Stage(output_rdd, shuffleDep, list(parent_stages), pipelines, dict(pipeline_edges), rdd_pipelines) + self.idToStage[stage.id] = stage + logger.debug('new stage: %s', stage) + return stage def getShuffleMapStage(self, dep): stage = self.shuffleToMapStage.get(dep.shuffleId, None) @@ -185,53 +453,138 @@ def getShuffleMapStage(self, dep): def getMissingParentStages(self, stage): missing = set() + + def _(r, dep): + if r.shouldCache and all(self.getCacheLocs(r)): + return False + + if isinstance(dep, ShuffleDependency): + stage = self.getShuffleMapStage(dep) + if not stage.isAvailable: + missing.add(stage) + return False + + return True + + walk_dependencies(stage.rdd, _) + return list(missing) + + def get_call_graph(self, final_rdd): + edges = Counter() # : count visited = set() - def visit(r): + to_visit = [final_rdd] + + while to_visit: + r = to_visit.pop(0) if r.id in visited: - return + continue visited.add(r.id) - if r.shouldCache and all(self.getCacheLocs(r)): - return - for dep in r.dependencies: - if isinstance(dep, ShuffleDependency): - stage = self.getShuffleMapStage(dep) - if not stage.isAvailable: - missing.add(stage) - elif isinstance(dep, NarrowDependency): - visit(dep.rdd) + to_visit.append(dep.rdd) + if dep.rdd.scope.api_callsite_id != r.scope.api_callsite_id: + edges[(dep.rdd.scope.api_callsite_id, r.scope.api_callsite_id)] += 1 + nodes = set() + run_scope = self.current_scope + edges[(final_rdd.scope.api_callsite_id, run_scope.api_callsite_id)] = 1 + for s, d in edges.keys(): + nodes.add(s) + nodes.add(d) + return sorted(list(nodes)), dict(edges) - visit(stage.rdd) - return list(missing) + @classmethod + def fmt_call_graph(cls, g0): + nodes0, edges0 = g0 + nodes = [] + edges = [{dag.KW_ID: "{}_{}".format(parent, child), dag.KW_SRC: parent, dag.KW_DST: child, "count": count} + for ((parent, child), count) in edges0.items()] + + for n in nodes0: + scope = Scope.scopes_by_api_callsite_id[n][0] + nodes.append({dag.KW_ID: n, + dag.KW_LABEL: scope.api, + dag.KW_DETAIL: [scope.api_callsite, scope.stack_above_api]}) + + return {dag.KW_NODES: nodes, dag.KW_EDGES: edges} + + def get_profs(self): + res = [marshal.loads(j) for j in self.jobstats] + running = self.get_running_prof() + if running: + res.append(marshal.loads(marshal.dumps(running))) + return res + + def get_running_prof(self): + if self.final_stage: + with self.final_lock: + return self._get_stats(self.final_rdd, self.final_stage) def runJob(self, finalRdd, func, partitions, allowLocal): + self.runJobTimes += 1 + self.current_scope = Scope.get("Job %d:{api}" % (self.runJobTimes, )) outputParts = list(partitions) numOutputParts = len(partitions) finalStage = self.newStage(finalRdd, None) - results = [None]*numOutputParts - finished = [None]*numOutputParts - lastFinished = 0 - numFinished = 0 + with self.final_lock: + self.final_rdd = finalRdd + self.final_stage = finalStage + try: + from dpark.web.ui.views.rddopgraph import StageInfo + stage_info = StageInfo() + stage_info.create_stage_info(finalStage) + + def create_stage_info_recur(cur_stage, is_final=False): + if not cur_stage or cur_stage.id in self.idToRunJob: + return + for par_stage in cur_stage.parents: + create_stage_info_recur(par_stage) + if cur_stage.id not in self.idToRunJob: + self.idToRunJob[cur_stage.id] = StageInfo.idToStageInfo[cur_stage.id] + self.idToRunJob[cur_stage.id].is_final = is_final + + create_stage_info_recur(finalStage, is_final=True) + except ImportError: + pass + results = [None] * numOutputParts + finished = [None] * numOutputParts + last_finished = 0 + finalStage.num_finished = 0 waiting = set() running = set() failed = set() - pendingTasks = {} + pendingTasks = {} # stage -> set([task_id..]) lastFetchFailureTime = 0 self.updateCacheLocs() - logger.debug("Final stage: %s, %d", finalStage, numOutputParts) - logger.debug("Parents of final stage: %s", finalStage.parents) - logger.debug("Missing parents: %s", self.getMissingParentStages(finalStage)) - - if allowLocal and (not finalStage.parents or not self.getMissingParentStages(finalStage)) and numOutputParts == 1: + logger.debug('Final stage: %s, %d', finalStage, numOutputParts) + logger.debug('Parents of final stage: %s', finalStage.parents) + logger.debug( + 'Missing parents: %s', + self.getMissingParentStages(finalStage)) + + def onStageFinished(stage): + def _(r, dep): + return r._do_checkpoint() + + MutableDict.merge() + walk_dependencies(stage.rdd, _) + logger.info("stage %d finish %s", stage.id, stage.fmt_stats()) + + if (allowLocal and + ( + not finalStage.parents or + not self.getMissingParentStages(finalStage) + ) and numOutputParts == 1): split = finalRdd.splits[outputParts[0]] yield func(finalRdd.iterator(split)) + onStageFinished(finalStage) return def submitStage(stage): - logger.debug("submit stage %s", stage) + if not stage.submit_time: + stage.submit_time = time.time() + logger.debug('submit stage %s', stage) if stage not in waiting and stage not in running: missing = self.getMissingParentStages(stage) if not missing: @@ -256,332 +609,563 @@ def submitMissingTasks(stage): have_prefer = False else: locs = [] - tasks.append(ResultTask(finalStage.id, finalRdd, - func, part, locs, i)) + tasks.append(ResultTask(finalStage.id, finalStage.try_id, part, finalRdd, + func, locs, i)) else: - for p in range(stage.numPartitions): - if not stage.outputLocs[p]: + for part in range(stage.numPartitions): + if not stage.outputLocs[part]: if have_prefer: - locs = self.getPreferredLocs(stage.rdd, p) + locs = self.getPreferredLocs(stage.rdd, part) if not locs: have_prefer = False else: locs = [] - tasks.append(ShuffleMapTask(stage.id, stage.rdd, - stage.shuffleDep, p, locs)) - logger.debug("add to pending %s tasks", len(tasks)) + tasks.append(ShuffleMapTask(stage.id, stage.try_id, part, stage.rdd, + stage.shuffleDep, locs)) + logger.debug('add to pending %s tasks', len(tasks)) myPending |= set(t.id for t in tasks) self.submitTasks(tasks) submitStage(finalStage) - while numFinished != numOutputParts: + while finalStage.num_finished != numOutputParts: try: evt = self.completionEvents.get(False) - except Queue.Empty: - self.check() - if self._shutdown: - sys.exit(1) - - if failed and time.time() > lastFetchFailureTime + RESUBMIT_TIMEOUT: + except queue.Empty: + if (failed and + time.time() > lastFetchFailureTime + RESUBMIT_TIMEOUT): self.updateCacheLocs() for stage in failed: - logger.info("Resubmitting failed stages: %s", stage) + logger.info('Resubmitting failed stages: %s', stage) submitStage(stage) failed.clear() else: time.sleep(0.1) continue + if evt is None: # aborted + for taskset in self.active_tasksets.values(): + self.tasksetFinished(taskset) + + if not self.is_dstream: + self._keep_stats(finalRdd, finalStage) + + raise RuntimeError('TaskSet aborted!') + task, reason = evt.task, evt.reason - stage = self.idToStage[task.stageId] - if stage not in pendingTasks: # stage from other job + stage = self.idToStage[task.stage_id] + if stage not in pendingTasks: # stage from other taskset continue - logger.debug("remove from pending %s from %s", task, stage) + logger.debug('remove from pending %s from %s', task, stage) pendingTasks[stage].remove(task.id) - if isinstance(reason, Success): + if reason == TaskEndReason.success: Accumulator.merge(evt.accumUpdates) + stage.task_stats[task.partition].append(evt.stats) if isinstance(task, ResultTask): finished[task.outputId] = True - numFinished += 1 + finalStage.num_finished += 1 results[task.outputId] = evt.result - while lastFinished < numOutputParts and finished[lastFinished]: - yield results[lastFinished] - results[lastFinished] = None - lastFinished += 1 + + while last_finished < numOutputParts and finished[last_finished]: + yield results[last_finished] + results[last_finished] = None + last_finished += 1 + + stage.finish() elif isinstance(task, ShuffleMapTask): - stage = self.idToStage[task.stageId] + stage = self.idToStage[task.stage_id] stage.addOutputLoc(task.partition, evt.result) - if not pendingTasks[stage] and all(stage.outputLocs): - logger.debug("%s finished; looking for newly runnable stages", stage) - MutableDict.merge() + if all(stage.outputLocs): + stage.finish() + logger.debug( + '%s finished; looking for newly runnable stages', + stage + ) + if pendingTasks[stage]: + logger.warn('dirty stage %d with %d tasks' + '(select at most 10 tasks:%s) not clean', + stage.id, len(pendingTasks[stage]), + str(list(pendingTasks[stage])[:10])) + del pendingTasks[stage] + onStageFinished(stage) running.remove(stage) - if stage.shuffleDep != None: - self.mapOutputTracker.registerMapOutputs( - stage.shuffleDep.shuffleId, - [l[-1] for l in stage.outputLocs]) + if stage.shuffleDep is not None: + MapOutputTracker.set_locs( + stage.shuffleDep.shuffleId, + [l[-1] for l in stage.outputLocs]) self.updateCacheLocs() - newlyRunnable = set(stage for stage in waiting if not self.getMissingParentStages(stage)) + newlyRunnable = set( + stage for stage in waiting + if not self.getMissingParentStages(stage) + ) waiting -= newlyRunnable running |= newlyRunnable - logger.debug("newly runnable: %s, %s", waiting, newlyRunnable) + logger.debug( + 'newly runnable: %s, %s', waiting, newlyRunnable) for stage in newlyRunnable: submitMissingTasks(stage) - elif isinstance(reason, FetchFailed): + elif reason == TaskEndReason.fetch_failed: + exception = evt.result if stage in running: waiting.add(stage) - mapStage = self.shuffleToMapStage[reason.shuffleId] - mapStage.removeHost(reason.serverUri) + running.remove(stage) + mapStage = self.shuffleToMapStage[exception.shuffleId] + mapStage.removeHost(exception.serverUri) failed.add(mapStage) lastFetchFailureTime = time.time() else: - logger.error("task %s failed: %s %s %s", task, reason, type(reason), reason.message) + logger.error( + 'task %s failed: %s %s %s', + task, + reason, + type(reason), + reason.message) raise Exception(reason.message) - MutableDict.merge() - assert not any(results) + onStageFinished(finalStage) + + if not self.is_dstream: + self._keep_stats(finalRdd, finalStage) + assert all(finished) + with self.final_lock: + self.final_stage = None + self.final_rdd = None return def getPreferredLocs(self, rdd, partition): return rdd.preferredLocations(rdd.splits[partition]) + def _keep_stats(self, final_rdd, final_stage): + try: + stats = self._get_stats(final_rdd, final_stage) + self.jobstats.append(marshal.dumps(stats)) + if self.loghub_dir: + self._dump_stats(stats) + except Exception as e: + logger.exception("Fail to dump job stats: %s.", e) + + def _dump_stats(self, stats): + name = "_".join(map(str, ['sched', self.id, "job", self.runJobTimes])) + ".json" + path = os.path.join(self.loghub_dir, name) + logger.info("writing profile to %s", path) + with open(path, 'w') as f: + json.dump(stats, f, indent=4) + + def _get_stats(self, final_rdd, final_stage): + call_graph = self.fmt_call_graph(self.get_call_graph(final_rdd)) + cmd = '[dpark] ' + \ + os.path.abspath(sys.argv[0]) + ' ' + ' '.join(sys.argv[1:]) + + stages = sorted([s.get_prof() for s in final_stage.get_tree_stages()], + key=lambda x: x['info']['start_time']) + + sink_scope = self.current_scope + sink_id = "SINK_{}_{}".format(self.id, self.runJobTimes) + sink_node = { + dag.KW_TYPE: "sink", + dag.KW_ID: sink_id, + dag.KW_LABEL: sink_scope.name, + "call_id": sink_scope.api_callsite_id + } + + sink_edge = { + "source": final_stage.get_node_id(final_stage.id, final_stage.rdd_pipelines[final_rdd.id]), + "target": sink_id, + } + run = {'framework': self.frameworkId, + 'scheduler': self.id, + "run": self.runJobTimes, + 'sink': { + "call_site": sink_scope.api_callsite, + "node": sink_node, + "edges": sink_edge, + }, + 'stages': stages, + "call_graph": call_graph, + } + + ret = { + 'script': { + 'cmd': cmd, + 'env': {'PWD': os.getcwd()} + }, + 'run': run + } + return ret + def run_task(task, aid): - logger.debug("Running task %r", task) + logger.debug('Running task %r', task) try: Accumulator.clear() result = task.run(aid) accumUpdates = Accumulator.values() MutableDict.flush() - return (task.id, Success(), result, accumUpdates) - except Exception, e: - logger.error("error in task %s", task) + return task.id, result, accumUpdates + except Exception as e: + logger.error('error in task %s', task) import traceback traceback.print_exc() - return (task.id, OtherFailure("exception:" + str(e)), None, None) + e.task_id = task.id + raise e class LocalScheduler(DAGScheduler): attemptId = 0 + def nextAttempId(self): self.attemptId += 1 return self.attemptId def submitTasks(self, tasks): - logger.debug("submit tasks %s in LocalScheduler", tasks) + logger.debug('submit tasks %s in LocalScheduler', tasks) for task in tasks: -# task = cPickle.loads(cPickle.dumps(task, -1)) - _, reason, result, update = run_task(task, self.nextAttempId()) - self.taskEnded(task, reason, result, update) + task_copy = cPickle.loads(cPickle.dumps(task, -1)) + try: + _, result, update = run_task(task_copy, self.nextAttempId()) + self.taskEnded(task, TaskEndReason.success, result, update) + except Exception: + self.taskEnded(task, TaskEndReason.other_failure, None, None) -def run_task_in_process(task, tid, environ): - from dpark.env import env - workdir = environ.get('WORKDIR') - environ['SERVER_URI'] = 'file://%s' % workdir[0] - env.start(False, environ) - logger.debug("run task in process %s %s", task, tid) +def run_task_in_process(task, tid, environ): try: - return run_task(task, tid) + return TaskEndReason.success, run_task(task, tid) except KeyboardInterrupt: sys.exit(0) + except Exception as e: + return TaskEndReason.other_failure, e + class MultiProcessScheduler(LocalScheduler): + def __init__(self, threads): LocalScheduler.__init__(self) self.threads = threads self.tasks = {} - self.pool = multiprocessing.Pool(self.threads or 2) + self.pool = None def submitTasks(self, tasks): if not tasks: return - logger.info("Got a job with %d tasks: %s", len(tasks), tasks[0].rdd) + logger.info('Got a taskset with %d tasks: %s', len(tasks), tasks[0].rdd) total, self.finished, start = len(tasks), 0, time.time() + + def initializer(): + # when called on subprocess of multiprocessing's Pool, + # default sighandler of SIGTERM will be called to quit gracefully, + # and we ignore other signals to prevent dead lock. + + import signal + from .context import _signals + for sig in _signals: + if sig == signal.SIGTERM: + signal.signal(sig, signal.SIG_DFL) + else: + signal.signal(sig, signal.SIG_IGN) + def callback(args): - logger.debug("got answer: %s", args) - tid, reason, result, update = args + state, data = args + logger.debug('task end: %s', state) + + if state == TaskEndReason.other_failure: + logger.warning('task failed: %s', data) + self.taskEnded(data.task_id, TaskEndReason.other_failure, result=None, accumUpdates=None) + return + + tid, result, update = data task = self.tasks.pop(tid) self.finished += 1 - logger.info("Task %s finished (%d/%d) \x1b[1A", - tid, self.finished, total) + logger.info('Task %s finished (%d/%d) \x1b[1A', + tid, self.finished, total) if self.finished == total: - logger.info("Job finished in %.1f seconds" + " "*20, time.time() - start) - self.taskEnded(task, reason, result, update) + logger.info( + 'TaskSet finished in %.1f seconds' + ' ' * 20, + time.time() - start) + self.taskEnded(task, TaskEndReason.success, result, update) for task in tasks: - logger.debug("put task async: %s", task) + logger.debug('put task async: %s', task) self.tasks[task.id] = task + if not self.pool: + # daemonic processes are not allowed to have children + from dpark.broadcast import start_download_manager + start_download_manager() + self.pool = multiprocessing.Pool( + self.threads or 2, + initializer=initializer + ) + self.pool.apply_async(run_task_in_process, - [task, self.nextAttempId(), env.environ], - callback=callback) + [task, self.nextAttempId(), env.environ], + callback=callback) def stop(self): - self.pool.terminate() - self.pool.join() - logger.debug("process pool stopped") - - -def profile(f): - def func(*args, **kwargs): - path = '/tmp/worker-%s.prof' % os.getpid() - import cProfile - import pstats - func = f - cProfile.runctx('func(*args, **kwargs)', - globals(), locals(), path) - stats = pstats.Stats(path) - stats.strip_dirs() - stats.sort_stats('time', 'calls') - stats.print_stats(20) - stats.sort_stats('cumulative') - stats.print_stats(20) - return func + if self.pool: + self.pool.terminate() + self.pool.join() + logger.debug('process pool stopped') + def safe(f): def _(self, *a, **kw): with self.lock: r = f(self, *a, **kw) return r + return _ -def int2ip(n): - return "%d.%d.%d.%d" % (n & 0xff, (n>>8)&0xff, (n>>16)&0xff, n>>24) + +class LogReceiver(object): + + def __init__(self, output): + self.output = output + self._started = False + self.addr = None + + def start(self): + ctx = zmq.Context() + sock = ctx.socket(zmq.PULL) + port = sock.bind_to_random_port('tcp://0.0.0.0') + self._started = True + + def collect_log(): + while self._started: + if sock.poll(1000, zmq.POLLIN): + line = sock.recv() + self.output.write(line) + + sock.close() + ctx.destroy() + + spawn(collect_log) + + host = socket.gethostname() + self.addr = 'tcp://%s:%d' % (host, port) + logger.debug('log collecter start at %s', self.addr) + + def stop(self): + self._started = False + class MesosScheduler(DAGScheduler): - def __init__(self, master, options): + def __init__(self, master, options, webui_url=None): DAGScheduler.__init__(self) self.master = master - self.use_self_as_exec = options.self self.cpus = options.cpus self.mem = options.mem - self.task_per_node = options.parallel or multiprocessing.cpu_count() + self.task_per_node = options.parallel or 8 self.group = options.group self.logLevel = options.logLevel self.options = options + self.role = options.role + self.color = options.color + self.webui_url = webui_url self.started = False self.last_finish_time = 0 + self.last_task_launch_time = None + self.is_suppressed = False self.isRegistered = False self.executor = None self.driver = None - self.out_logger = None - self.err_logger = None + self.out_logger = LogReceiver(sys.stdout) + self.err_logger = LogReceiver(sys.stderr) self.lock = threading.RLock() - self.init_job() + self.task_host_manager = TaskHostManager() + self.init_tasksets() - def init_job(self): - self.activeJobs = {} - self.activeJobsQueue = [] - self.taskIdToJobId = {} - self.taskIdToSlaveId = {} - self.jobTasks = {} - self.slaveTasks = {} - self.slaveFailed = {} + def init_tasksets(self): + self.active_tasksets = {} + self.ttid_to_agent_id = {} + self.agent_id_to_ttids = {} def clear(self): DAGScheduler.clear(self) - self.init_job() + self.init_tasksets() + + def processHeartBeat(self): + # no need in dpark now, just for compatibility with pymesos + pass def start(self): - if not self.out_logger: - self.out_logger = self.start_logger(sys.stdout) - if not self.err_logger: - self.err_logger = self.start_logger(sys.stderr) + self.out_logger.start() + self.err_logger.start() def start_driver(self): - name = '[dpark] ' + os.path.abspath(sys.argv[0]) + ' ' + ' '.join(sys.argv[1:]) + name = '[dpark] ' + \ + os.path.abspath(sys.argv[0]) + ' ' + ' '.join(sys.argv[1:]) if len(name) > 256: name = name[:256] + '...' - framework = mesos_pb2.FrameworkInfo() + framework = Dict() framework.user = getuser() if framework.user == 'root': - raise Exception("dpark is not allowed to run as 'root'") + raise Exception('dpark is not allowed to run as \'root\'') framework.name = name + if self.role: + framework.role = self.role framework.hostname = socket.gethostname() + if self.webui_url: + framework.webui_url = self.webui_url - self.driver = mesos.MesosSchedulerDriver(self, framework, - self.master) + self.driver = MesosSchedulerDriver( + self, framework, self.master, use_addict=True + ) self.driver.start() - logger.debug("Mesos Scheudler driver started") + logger.debug('Mesos Scheudler driver started') self.started = True self.last_finish_time = time.time() + def check(): while self.started: - now = time.time() - if not self.activeJobs and now - self.last_finish_time > MAX_IDLE_TIME: - logger.info("stop mesos scheduler after %d seconds idle", - now - self.last_finish_time) - self.stop() - break + with self.lock: + now = time.time() + if (not self.active_tasksets and + now - self.last_finish_time > MAX_IDLE_TIME): + logger.info('stop mesos scheduler after %d seconds idle', + now - self.last_finish_time) + self.stop() + break + + for taskset in self.active_tasksets.values(): + if taskset.check_task_timeout(): + self.requestMoreResources() time.sleep(1) spawn(check) - def start_logger(self, output): - sock = env.ctx.socket(zmq.PULL) - port = sock.bind_to_random_port("tcp://0.0.0.0") - - def collect_log(): - while not self._shutdown: - if sock.poll(1000, zmq.POLLIN): - line = sock.recv() - output.write(line) - - spawn(collect_log) - - host = socket.gethostname() - addr = "tcp://%s:%d" % (host, port) - logger.debug("log collecter start at %s", addr) - return addr - @safe def registered(self, driver, frameworkId, masterInfo): self.isRegistered = True - logger.debug("connect to master %s:%s(%s), registered as %s", - int2ip(masterInfo.ip), masterInfo.port, masterInfo.id, - frameworkId.value) + self.frameworkId = frameworkId.value + logger.debug('connect to master %s:%s, registered as %s', + masterInfo.hostname, masterInfo.port, frameworkId.value) self.executor = self.getExecutorInfo(str(frameworkId.value)) + from dpark.utils.log import add_loghub + _, self.loghub_dir = add_loghub(self.frameworkId) @safe def reregistered(self, driver, masterInfo): - logger.warning("re-connect to mesos master %s:%s(%s)", - int2ip(masterInfo.ip), masterInfo.port, masterInfo.id) + logger.warning('re-connect to mesos master %s:%s', + masterInfo.hostname, masterInfo.port) - @safe def disconnected(self, driver): - logger.debug("framework is disconnected") + logger.debug('framework is disconnected') + + def _get_container_image(self): + return self.options.image @safe def getExecutorInfo(self, framework_id): - info = mesos_pb2.ExecutorInfo() - if hasattr(info, 'framework_id'): - info.framework_id.value = framework_id + info = Dict() + info.framework_id.value = framework_id + info.command.value = '%s %s' % ( + sys.executable, + os.path.abspath( + os.path.join( + os.path.dirname(__file__), + 'executor.py')) + ) + info.executor_id.value = env.get('DPARK_ID', 'default') + info.command.environment.variables = variables = [] + info.shutdown_grace_period.nanoseconds = 5 * (10 ** 9) + + v = Dict() + variables.append(v) + v.name = 'UID' + v.value = str(os.getuid()) + + v = Dict() + variables.append(v) + v.name = 'GID' + v.value = str(os.getgid()) + + container_image = self._get_container_image() + if container_image: + info.container.type = 'DOCKER' + info.container.docker.image = container_image + info.container.docker.parameters = parameters = [] + p = Dict() + p.key = 'memory-swap' + p.value = '-1' + parameters.append(p) + + info.container.volumes = volumes = [] + for path in ['/etc/passwd', '/etc/group']: + v = Dict() + volumes.append(v) + v.host_path = v.container_path = path + v.mode = 'RO' + + for path in conf.MOOSEFS_MOUNT_POINTS: + v = Dict() + volumes.append(v) + v.host_path = v.container_path = path + v.mode = 'RW' + + for path in conf.DPARK_WORK_DIR.split(','): + v = Dict() + volumes.append(v) + v.host_path = v.container_path = path + v.mode = 'RW' + + def _mount_volume(_volumes, _host_path, _container_path, _mode): + _v = Dict() + _volumes.append(_v) + _v.container_path = _container_path + _v.mode = _mode + if _host_path: + _v.host_path = _host_path + + if self.options.volumes: + for volume in self.options.volumes.split(','): + fields = volume.split(':') + if len(fields) == 3: + host_path, container_path, mode = fields + mode = mode.upper() + assert mode in ('RO', 'RW') + elif len(fields) == 2: + host_path, container_path = fields + mode = 'RW' + elif len(fields) == 1: + container_path, = fields + host_path = '' + mode = 'RW' + else: + raise Exception('cannot parse volume %s', volume) + _mount_volume(volumes, host_path, + container_path, mode) - if self.use_self_as_exec: - info.command.value = os.path.abspath(sys.argv[0]) - info.executor_id.value = sys.argv[0] - else: - info.command.value = '%s %s' % ( - sys.executable, - os.path.abspath(os.path.join(os.path.dirname(__file__), 'executor.py')) - ) - info.executor_id.value = "default" + info.resources = resources = [] - mem = info.resources.add() + mem = Dict() + resources.append(mem) mem.name = 'mem' - mem.type = 0 #mesos_pb2.Value.SCALAR + mem.type = 'SCALAR' mem.scalar.value = EXECUTOR_MEMORY + + cpus = Dict() + resources.append(cpus) + cpus.name = 'cpus' + cpus.type = 'SCALAR' + cpus.scalar.value = EXECUTOR_CPUS + Script = os.path.realpath(sys.argv[0]) - if hasattr(info, 'name'): - info.name = Script + info.name = Script - info.data = marshal.dumps((Script, os.getcwd(), sys.path, dict(os.environ), - self.task_per_node, self.out_logger, self.err_logger, self.logLevel, env.environ)) + info.data = encode_data(marshal.dumps( + ( + Script, os.getcwd(), sys.path, dict(os.environ), + self.task_per_node, self.out_logger.addr, self.err_logger.addr, + self.logLevel, self.color, env.environ + ) + )) + assert len(info.data) < (50 << 20), \ + 'Info data too large: %s' % (len(info.data),) return info @safe @@ -589,11 +1173,30 @@ def submitTasks(self, tasks): if not tasks: return - job = SimpleJob(self, tasks, self.cpus, tasks[0].rdd.mem or self.mem) - self.activeJobs[job.id] = job - self.activeJobsQueue.append(job) - self.jobTasks[job.id] = set() - logger.info("Got job %d with %d tasks: %s", job.id, len(tasks), tasks[0].rdd) + rdd = tasks[0].rdd + assert all(t.rdd is rdd for t in tasks) + + taskset = TaskSet(self, tasks, rdd.cpus or self.cpus, rdd.mem or self.mem, + rdd.gpus, self.task_host_manager) + self.active_tasksets[taskset.id] = taskset + stage_scope = '' + try: + from dpark.web.ui.views.rddopgraph import StageInfo + stage_scope = StageInfo.idToRDDNode[tasks[0].rdd.id].scope.api_callsite + except: + pass + + stage = self.idToStage[tasks[0].stage_id] + stage.num_try += 1 + stage.taskcounters.append(taskset.counter) + logger.info( + 'Got taskset %s with %d tasks for stage: %d ' + 'at scope[%s] and rdd:%s', + taskset.id, + len(tasks), + tasks[0].stage_id, + stage_scope, + tasks[0].rdd) need_revive = self.started if not self.started: @@ -607,218 +1210,326 @@ def submitTasks(self, tasks): self.requestMoreResources() def requestMoreResources(self): - logger.debug("reviveOffers") + logger.debug('reviveOffers') self.driver.reviveOffers() + self.is_suppressed = False @safe def resourceOffers(self, driver, offers): - rf = mesos_pb2.Filters() - if not self.activeJobs: + rf = Dict() + now = time.time() + if not self.active_tasksets or (all(taskset.counter.launched == taskset.counter.n + for taskset in self.active_tasksets.values()) + and self.last_task_launch_time is not None + and self.last_task_launch_time + conf.TIME_TO_SUPPRESS < now): + logger.debug('suppressOffers') + driver.suppressOffers() + self.is_suppressed = True rf.refuse_seconds = 60 * 5 for o in offers: - driver.launchTasks(o.id, [], rf) + driver.declineOffer(o.id, rf) return start = time.time() - random.shuffle(offers) + filter_offer = [] + for o in offers: + try: + if conf.ban(o.hostname): + logger.debug("skip offer on banned node: %s", o.hostname) + driver.declineOffer(o.id, filters=Dict(refuse_seconds=0xFFFFFFFF)) + continue + except: + logger.exception("bad ban() func in dpark.conf") + + group = ( + self.getAttribute( + o.attributes, + 'group') or 'None') + if (self.group or group.startswith( + '_')) and group not in self.group: + driver.declineOffer(o.id, filters=Dict(refuse_seconds=0xFFFFFFFF)) + continue + unavailability = o.get('unavailability') + if (unavailability is not None and + sec2nanosec(time.time() + conf.DEFAULT_TASK_TIME) >= unavailability['start']['nanoseconds']): + logger.debug('the host %s plan to maintain, so skip it', o.hostname) + driver.declineOffer(o.id, filters=Dict(refuse_seconds=600)) + continue + if self.task_host_manager.is_unhealthy_host(o.hostname): + logger.warning('the host %s is unhealthy so skip it', o.hostname) + driver.declineOffer(o.id, filters=Dict(refuse_seconds=1800)) + continue + self.task_host_manager.register_host(o.hostname) + filter_offer.append(o) + offers = filter_offer cpus = [self.getResource(o.resources, 'cpus') for o in offers] + gpus = [self.getResource(o.resources, 'gpus') for o in offers] mems = [self.getResource(o.resources, 'mem') - - (o.slave_id.value not in self.slaveTasks - and EXECUTOR_MEMORY or 0) + - (o.agent_id.value not in self.agent_id_to_ttids + and EXECUTOR_MEMORY or 0) for o in offers] - logger.debug("get %d offers (%s cpus, %s mem), %d jobs", - len(offers), sum(cpus), sum(mems), len(self.activeJobs)) + # logger.debug('get %d offers (%s cpus, %s mem, %s gpus), %d tasksets', + # len(offers), sum(cpus), sum(mems), sum(gpus), len(self.active_tasksets)) + mesos_tasks = {} tasks = {} - for job in self.activeJobsQueue: + max_create_time = 0 + for taskset in self.active_tasksets.values(): while True: - launchedTask = False - for i,o in enumerate(offers): - sid = o.slave_id.value - if self.group and (self.getAttribute(o.attributes, 'group') or 'none') not in self.group: + host_offers = {} + for i, o in enumerate(offers): + if self.agent_id_to_ttids.get(o.agent_id.value, 0) >= self.task_per_node: + logger.debug('the task limit exceeded at host %s', + o.hostname) continue - if self.slaveFailed.get(sid, 0) >= MAX_FAILED: + if (mems[i] < self.mem + EXECUTOR_MEMORY + or cpus[i] < self.cpus + EXECUTOR_CPUS): continue - if self.slaveTasks.get(sid, 0) >= self.task_per_node: - continue - if mems[i] < self.mem or cpus[i]+1e-4 < self.cpus: - continue - t = job.slaveOffer(str(o.hostname), cpus[i], mems[i]) - if not t: - continue - task = self.createTask(o, job, t, cpus[i]) - tasks.setdefault(o.id.value, []).append(task) - - logger.debug("dispatch %s into %s", t, o.hostname) - tid = task.task_id.value - self.jobTasks[job.id].add(tid) - self.taskIdToJobId[tid] = job.id - self.taskIdToSlaveId[tid] = sid - self.slaveTasks[sid] = self.slaveTasks.get(sid, 0) + 1 + host_offers[o.hostname] = (i, o) + assigned_list = taskset.taskOffer(host_offers, cpus, mems, gpus) + if not assigned_list: + break + + for i, o, t in assigned_list: + t0 = time.time() + mesos_task = self.createTask(o, t) + max_create_time = max(max_create_time, time.time() - t0) + mesos_tasks.setdefault(o.id.value, []).append(mesos_task) + tasks.setdefault(o.id.value, []).append(t) + logger.debug('dispatch %s into %s', t, o.hostname) + ttid = mesos_task.task_id.value + agent_id = o.agent_id.value + taskset.ttids.add(ttid) + self.ttid_to_agent_id[ttid] = agent_id + self.agent_id_to_ttids[agent_id] = self.agent_id_to_ttids.get(agent_id, 0) + 1 cpus[i] -= min(cpus[i], t.cpus) mems[i] -= t.mem - launchedTask = True - - if not launchedTask: - break + gpus[i] -= t.gpus used = time.time() - start if used > 10: - logger.error("use too much time in slaveOffer: %.2fs", used) + logger.warning('use too much time in resourceOffers: %.2fs, %d offers,' + 'assigned %d tasks, max_create_time = %ds', + used, + len(offers), + len(mesos_tasks), + max_create_time) - rf.refuse_seconds = 5 for o in offers: - driver.launchTasks(o.id, tasks.get(o.id.value, []), rf) + oid = o.id.value + if oid in mesos_tasks: + driver.launchTasks(o.id, mesos_tasks[oid]) + for task in tasks[oid]: + task.stage_time = time.time() + else: + driver.declineOffer(o.id) + + if tasks: + self.last_task_launch_time = time.time() - logger.debug("reply with %d tasks, %s cpus %s mem left", - sum(len(ts) for ts in tasks.values()), sum(cpus), sum(mems)) + # logger.debug('reply with %d tasks, %s cpus %s mem %s gpus left', + # sum(len(ts) for ts in tasks.values()), + # sum(cpus), sum(mems), sum(gpus)) + + @safe + def inverseOffers(self, driver, offers): + for o in offers: + driver.acceptInverseOffers(o.id) @safe def offerRescinded(self, driver, offer_id): - logger.debug("rescinded offer: %s", offer_id) - if self.activeJobs: + logger.debug('rescinded offer: %s', offer_id) + if self.active_tasksets: self.requestMoreResources() def getResource(self, res, name): for r in res: if r.name == name: return r.scalar.value + return 0.0 def getAttribute(self, attrs, name): for r in attrs: if r.name == name: return r.text.value - def createTask(self, o, job, t, available_cpus): - task = mesos_pb2.TaskInfo() - tid = "%s:%s:%s" % (job.id, t.id, t.tried) - task.name = "task %s" % tid + def createTask(self, o, t): + task = Dict() + tid = t.try_id + task.name = 'task %s' % tid task.task_id.value = tid - task.slave_id.value = o.slave_id.value - task.data = compress(cPickle.dumps((t, t.tried), -1)) - task.executor.MergeFrom(self.executor) - if len(task.data) > 1000*1024: - logger.warning("task too large: %s %d", - t, len(task.data)) - - cpu = task.resources.add() + task.agent_id.value = o.agent_id.value + task.data = encode_data( + compress(cPickle.dumps((t, tid), -1)) + ) + task.executor = self.executor + if len(task.data) > 1000 * 1024: + logger.warning('task too large: %s %d', + t, len(task.data)) + + assert len(task.data) < (50 << 20), \ + 'Task data too large: %s' % (len(task.data),) + + resources = task.resources = [] + + cpu = Dict() + resources.append(cpu) cpu.name = 'cpus' - cpu.type = 0 #mesos_pb2.Value.SCALAR - cpu.scalar.value = min(t.cpus, available_cpus) - mem = task.resources.add() + cpu.type = 'SCALAR' + cpu.scalar.value = t.cpus + + mem = Dict() + resources.append(mem) mem.name = 'mem' - mem.type = 0 #mesos_pb2.Value.SCALAR + mem.type = 'SCALAR' mem.scalar.value = t.mem + + gpu = Dict() + resources.append(gpu) + gpu.name = 'gpus' + gpu.type = 'SCALAR' + gpu.scalar.value = t.gpus + return task @safe def statusUpdate(self, driver, status): - tid = status.task_id.value + + def plot_progresses(): + if self.color: + total = len(self.active_tasksets) + logger.info('\x1b[2K\x1b[J\x1b[1A') + for i, taskset_id in enumerate(self.active_tasksets): + if i == total - 1: + ending = '\x1b[%sA' % total + else: + ending = '' + + tasksets = self.active_tasksets[taskset_id] + tasksets.progress(ending) + + mesos_task_id = status.task_id.value state = status.state - logger.debug("status update: %s %s", tid, state) + source = status.source + reason = status.get('reason') - jid = self.taskIdToJobId.get(tid) - if jid not in self.activeJobs: - logger.debug("Ignoring update from TID %s " + - "because its job is gone", tid) - return + msg = status.get('message') # type: str + if source == 'SOURCE_EXECUTOR' and msg: + reason, msg = msg.split(':', 1) - job = self.activeJobs[jid] - _, task_id, tried = map(int, tid.split(':')) - if state == mesos_pb2.TASK_RUNNING: - return job.statusUpdate(task_id, tried, state) + data = status.get('data') + if data is not None: + data = cPickle.loads(decode_data(data)) - del self.taskIdToJobId[tid] - self.jobTasks[jid].remove(tid) - slave_id = self.taskIdToSlaveId[tid] - if slave_id in self.slaveTasks: - self.slaveTasks[slave_id] -= 1 - del self.taskIdToSlaveId[tid] + logger.debug('status update: %s %s %s %s', mesos_task_id, state, reason, msg) - if state in (mesos_pb2.TASK_FINISHED, mesos_pb2.TASK_FAILED) and status.data: - try: - reason,result,accUpdate = cPickle.loads(status.data) - if result: - flag, data = result - if flag >= 2: - try: - data = urllib.urlopen(data).read() - except IOError: - # try again - data = urllib.urlopen(data).read() - flag -= 2 - data = decompress(data) - if flag == 0: - result = marshal.loads(data) - else: - result = cPickle.loads(data) - except Exception, e: - logger.warning("error when cPickle.loads(): %s, data:%s", e, len(status.data)) - state = mesos_pb2.TASK_FAILED - return job.statusUpdate(task_id, tried, mesos_pb2.TASK_FAILED, 'load failed: %s' % e) + ttid = TTID(mesos_task_id) + + taskset = self.active_tasksets.get(ttid.taskset_id) # type: TaskSet + + if taskset is None: + if state == TaskState.running: + logger.debug('kill task %s as its taskset has gone', mesos_task_id) + self.driver.killTask(Dict(value=mesos_task_id)) else: - return job.statusUpdate(task_id, tried, state, - reason, result, accUpdate) - - # killed, lost, load failed - job.statusUpdate(task_id, tried, state, status.data) - #if state in (mesos_pb2.TASK_FAILED, mesos_pb2.TASK_LOST): - # self.slaveFailed[slave_id] = self.slaveFailed.get(slave_id,0) + 1 - - def jobFinished(self, job): - logger.debug("job %s finished", job.id) - if job.id in self.activeJobs: - del self.activeJobs[job.id] - self.activeJobsQueue.remove(job) - for id in self.jobTasks[job.id]: - del self.taskIdToJobId[id] - del self.taskIdToSlaveId[id] - del self.jobTasks[job.id] - self.last_finish_time = time.time() + logger.debug('ignore task %s as its taskset has gone', mesos_task_id) + return + + if mesos_task_id not in taskset.ttids: + logger.debug('ignore task %s as it has finished or failed, new msg: %s', mesos_task_id, (state, reason)) + return + + if state == TaskState.running: + taskset.statusUpdate(ttid.task_id, ttid.task_try, state) + if taskset.counter.finished == 0: + plot_progresses() + return - if not self.activeJobs: - self.slaveTasks.clear() - self.slaveFailed.clear() + # terminal state + taskset.ttids.discard(mesos_task_id) + if mesos_task_id in self.ttid_to_agent_id: + agent_id = self.ttid_to_agent_id[mesos_task_id] + if agent_id in self.agent_id_to_ttids: + self.agent_id_to_ttids[agent_id] -= 1 + del self.ttid_to_agent_id[mesos_task_id] + + if state == TaskState.finished: + try: + result, accUpdate, task_stats = data + flag, data = result + if flag >= 2: + try: + data = urllib.request.urlopen(data).read() + except IOError: + # try again + data = urllib.request.urlopen(data).read() + flag -= 2 + data = decompress(data) + if flag == 0: + result = marshal.loads(data) + else: + result = cPickle.loads(data) + taskset.statusUpdate(ttid.task_id, ttid.task_try, state, + result=result, update=accUpdate, stats=task_stats) + plot_progresses() + except Exception as e: + logger.warning('error when cPickle.loads(): %s, data:%s', e, len(data)) + state = TaskState.failed + taskset.statusUpdate(ttid.task_id, ttid.task_try, state, + reason=TaskEndReason.load_failed, message='load failed: %s' % e) + else: + exception = data if source == 'SOURCE_EXECUTOR' else None # type: Optional[Exception] + taskset.statusUpdate(ttid.task_id, ttid.task_try, state, reason, msg, result=exception) @safe - def check(self): - for job in self.activeJobs.values(): - if job.check_task_timeout(): - self.requestMoreResources() + def tasksetFinished(self, taskset): + logger.debug('taskset %s finished', taskset.id) + if taskset.id in self.active_tasksets: + self.last_finish_time = time.time() + for mesos_task_id in taskset.ttids: + self.driver.killTask(Dict(value=mesos_task_id)) + del self.active_tasksets[taskset.id] + if not self.active_tasksets: + self.agent_id_to_ttids.clear() @safe - def error(self, driver, code, message): - logger.warning("Mesos error message: %s (code: %s)", message, code) - #if self.activeJobs: - # self.requestMoreResources() + def error(self, driver, message): + logger.error('Mesos error message: %s', message) + raise RuntimeError(message) - #@safe + # @safe def stop(self): if not self.started: return - logger.debug("stop scheduler") + logger.debug('stop scheduler') self.started = False self.isRegistered = False self.driver.stop(False) + self.driver.join() self.driver = None + self.out_logger.stop() + self.err_logger.stop() + def defaultParallelism(self): return 16 - def frameworkMessage(self, driver, slave, executor, data): - logger.warning("[slave %s] %s", slave.value, data) + def frameworkMessage(self, driver, executor_id, agent_id, data): + logger.warning('[agent %s] %s', agent_id.value, data) - def executorLost(self, driver, executorId, slaveId, status): - logger.warning("executor at %s %s lost: %s", slaveId.value, executorId.value, status) - self.slaveTasks.pop(slaveId.value, None) - self.slaveFailed.pop(slaveId.value, None) + def executorLost(self, driver, executor_id, agent_id, status): + logger.warning( + 'executor at %s %s lost: %s', + agent_id.value, + executor_id.value, + status) + self.agent_id_to_ttids.pop(agent_id.value, None) - def slaveLost(self, driver, slaveId): - logger.warning("slave %s lost", slaveId.value) - self.slaveTasks.pop(slaveId.value, None) - self.slaveFailed.pop(slaveId.value, None) + def slaveLost(self, driver, agent_id): + logger.warning('agent %s lost', agent_id.value) + self.agent_id_to_ttids.pop(agent_id.value, None) - def killTask(self, job_id, task_id, tried): - tid = mesos_pb2.TaskID() - tid.value = "%s:%s:%s" % (job_id, task_id, tried) + def killTask(self, task_id, num_try): + tid = Dict() + tid.value = TTID.make_ttid(task_id, num_try) self.driver.killTask(tid) diff --git a/dpark/serialize.py b/dpark/serialize.py index 7489a4ec..0b3004ce 100644 --- a/dpark/serialize.py +++ b/dpark/serialize.py @@ -1,12 +1,142 @@ -import sys, types -from cStringIO import StringIO -import marshal, new, cPickle +from __future__ import absolute_import +from __future__ import print_function +import sys +import types +import marshal +import types +import six import itertools -from pickle import Pickler, whichmodule -import logging -logger = logging.getLogger(__name__) +from collections import deque +from functools import partial +from six.moves import range, cPickle +from six import int2byte +from dpark.utils.log import get_logger +from pickle import whichmodule, PROTO, STOP + +if six.PY2: + from pickle import Pickler +else: + from pickle import _Pickler as Pickler + +try: + from cStringIO import StringIO +except ImportError: + from six import BytesIO as StringIO + +logger = get_logger(__name__) + + +class LazySave(object): + """Out of band marker for lazy saves among lazy writes.""" + + __slots__ = ['obj'] + + def __init__(self, obj): + self.obj = obj + + def __repr__(self): + return '' % repr(self.obj) + + +class LazyMemo(object): + """Out of band marker for lazy memos among lazy writes.""" + + __slots__ = ['obj'] + + def __init__(self, obj): + self.obj = obj + + def __repr__(self): + return '' % repr(self.obj) + class MyPickler(Pickler): + def __init__(self, file, protocol=None): + Pickler.__init__(self, file, protocol) + self.lazywrites = deque() + self.realwrite = file.write + + # Pickler.__init__ overwrites self.write, we do not want that + del self.write + + def write(self, *args): + if self.lazywrites: + self.lazywrites.append(args) + else: + self.realwrite(*args) + + def save(self, obj): + self.lazywrites.append(LazySave(obj)) + + def realsave(self, obj): + def _name(obj): + try: + name = getattr(obj, '__name__', None) + if name is not None: + return ': %s' % name + except Exception: + pass + + return '' + + def _loc(obj): + try: + fn = getattr(obj, '__file__', None) + if fn is not None: + return ' @%s' % (fn,) + + obj = getattr(obj, 'im_func', obj) + code = getattr(obj, '__code__', None) + if code is not None: + return ' @%s:%s' % (code.co_filename, code.co_firstlineno) + except Exception: + pass + + return '' + + try: + Pickler.save(self, obj) + except TypeError: + logger.error('Failed to serialize %s%s%s', + type(obj), _name(obj), _loc(obj)) + raise + + def lazymemoize(self, obj): + """Store an object in the memo.""" + if self.lazywrites: + self.lazywrites.append(LazyMemo(obj)) + else: + self.realmemoize(obj) + + memoize = lazymemoize + + realmemoize = Pickler.memoize + + def dump(self, obj): + """Write a pickled representation of obj to the open file.""" + if self.proto >= 2: + self.write(PROTO + int2byte(self.proto)) + self.realsave(obj) + queues = deque([self.lazywrites]) + while queues: + lws = queues[0] + self.lazywrites = deque() + while lws: + lw = lws.popleft() + if isinstance(lw, LazySave): + self.realsave(lw.obj) + if self.lazywrites: + queues.appendleft(self.lazywrites) + break + elif isinstance(lw, LazyMemo): + self.realmemoize(lw.obj) + else: + self.realwrite(*lw) + else: + queues.popleft() + + self.realwrite(STOP) + dispatch = Pickler.dispatch.copy() @classmethod @@ -17,42 +147,54 @@ def dispatcher(self, obj): self.save_global(obj, rv) else: self.save_reduce(obj=obj, *rv) + cls.dispatch[type] = dispatcher + def dumps(o): io = StringIO() MyPickler(io, -1).dump(o) return io.getvalue() + def loads(s): return cPickle.loads(s) + dump_func = dumps load_func = loads + def reduce_module(mod): - return load_module, (mod.__name__, ) + return load_module, (mod.__name__,) + def load_module(name): __import__(name) return sys.modules[name] + MyPickler.register(types.ModuleType, reduce_module) + class RecursiveFunctionPlaceholder(object): """ Placeholder for a recursive reference to the current function, to avoid infinite recursion when serializing recursive functions. """ + def __eq__(self, other): return isinstance(other, RecursiveFunctionPlaceholder) + RECURSIVE_FUNCTION_PLACEHOLDER = RecursiveFunctionPlaceholder() + def marshalable(o): - if o is None: return True + if o is None: + return True t = type(o) - if t in (str, unicode, bool, int, long, float, complex): + if t in (six.binary_type, six.text_type, bool, int, int, float, complex): return True if t in (tuple, list, set): for i in itertools.islice(o, 100): @@ -60,19 +202,22 @@ def marshalable(o): return False return True if t == dict: - for k,v in itertools.islice(o.iteritems(), 100): + for k, v in itertools.islice(six.iteritems(o), 100): if not marshalable(k) or not marshalable(v): return False return True return False + OBJECT_SIZE_LIMIT = 100 << 10 + def create_broadcast(name, obj, func_name): - import dpark + from dpark.broadcast import Broadcast logger.info("use broadcast for object %s %s (used in function %s)", - name, type(obj), func_name) - return dpark._ctx.broadcast(obj) + name, type(obj), func_name) + return Broadcast(obj) + def dump_obj(f, name, obj): if obj is f: @@ -92,6 +237,7 @@ def dump_obj(f, name, obj): logger.warning("broadcast of %s obj too large", type(obj)) return b + def get_co_names(code): co_names = code.co_names for const in code.co_consts: @@ -100,44 +246,69 @@ def get_co_names(code): return co_names -def dump_closure(f): - code = f.func_code + +def dump_closure(f, skip=set()): + def _do_dump(f): + for i, c in enumerate(f.__closure__): + try: + if hasattr(c, 'cell_contents'): + yield dump_obj(f, 'cell%d' % i, c.cell_contents) + else: + yield None + except ValueError: + yield None + + code = f.__code__ glob = {} for n in get_co_names(code): - r = f.func_globals.get(n) - if r is not None: + r = f.__globals__.get(n) + if r is not None and n not in skip: glob[n] = dump_obj(f, n, r) closure = None - if f.func_closure: - closure = tuple(dump_obj(f, 'cell%d' % i, c.cell_contents) - for i, c in enumerate(f.func_closure)) - return marshal.dumps((code, glob, f.func_name, f.func_defaults, closure, f.__module__)) + if f.__closure__: + closure = tuple(_do_dump(f)) + return marshal.dumps( + (code, glob, f.__name__, f.__defaults__, closure, f.__module__)) + def load_closure(bytes): code, glob, name, defaults, closure, mod = marshal.loads(bytes) - glob = dict((k, loads(v)) for k,v in glob.items()) + glob = dict((k, loads(v)) for k, v in glob.items()) glob['__builtins__'] = __builtins__ - closure = closure and reconstruct_closure([loads(c) for c in closure]) or None - f = new.function(code, glob, name, defaults, closure) + closure = closure and reconstruct_closure(closure) or None + f = types.FunctionType(code, glob, name, defaults, closure) f.__module__ = mod - # Replace the recursive function placeholders with this simulated function pointer + # Replace the recursive function placeholders with this simulated function + # pointer for key, value in glob.items(): if RECURSIVE_FUNCTION_PLACEHOLDER == value: - f.func_globals[key] = f + f.__globals__[key] = f return f + def make_cell(value): - return (lambda: value).func_closure[0] + return (lambda: value).__closure__[0] + + +def make_empty_cell(): + if False: + unreachable = None + return (lambda: unreachable).__closure__[0] + + +def reconstruct_closure(closure): + return tuple( + [make_cell(loads(v)) if v is not None + else make_empty_cell() for v in closure]) -def reconstruct_closure(values): - return tuple([make_cell(v) for v in values]) def get_global_function(module, name): __import__(module) mod = sys.modules[module] return getattr(mod, name) + def reduce_function(obj): name = obj.__name__ if not name or name == '': @@ -147,8 +318,9 @@ def reduce_function(obj): if module is None: module = whichmodule(obj, name) - if module == '__main__' and name not in ('load_closure','load_module', - 'load_method', 'load_local_class'): # fix for test + if module == '__main__' and \ + name not in ('load_closure', 'load_module', + 'load_method', 'load_local_class'): # fix for test return load_closure, (dump_closure(obj),) try: @@ -160,12 +332,21 @@ def reduce_function(obj): return load_closure, (dump_closure(obj),) return name + classes_dumping = set() internal_fields = { '__weakref__': False, '__dict__': False, - '__doc__': True + '__doc__': True, + '__slots__': True, } + +member_descripter_types = ( + types.MemberDescriptorType, + type(LazySave.obj) +) + + def dump_local_class(cls): name = cls.__name__ if cls in classes_dumping: @@ -174,17 +355,28 @@ def dump_local_class(cls): classes_dumping.add(cls) internal = {} external = {} - for k in cls.__dict__: + keys = list(cls.__dict__.keys()) + for k in keys: if k not in internal_fields: - v = getattr(cls, k) + v = cls.__dict__[k] if isinstance(v, property): k = ('property', k) v = (v.fget, v.fset, v.fdel, v.__doc__) - if isinstance(v, types.FunctionType): + if isinstance(v, staticmethod): k = ('staticmethod', k) + v = dump_closure(v.__func__, skip=set(keys)) + + if isinstance(v, classmethod): + k = ('classmethod', k) + v = dump_closure(v.__func__, skip=set(keys)) + + if isinstance(v, types.FunctionType): + k = ('method', k) + v = dump_closure(v, skip=set(keys)) - external[k] = v + if not isinstance(v, member_descripter_types): + external[k] = v elif internal_fields[k]: internal[k] = getattr(cls, k) @@ -195,7 +387,10 @@ def dump_local_class(cls): return result + classes_loaded = {} + + def load_local_class(bytes): t = loads(bytes) if not isinstance(t, tuple): @@ -205,9 +400,15 @@ def load_local_class(bytes): if name in classes_loaded: return classes_loaded[name] - cls = type(name, bases, internal) + if any(isinstance(base, type) for base in bases): + cls = type(name, bases, internal) + else: + assert six.PY2 + cls = types.ClassType(name, bases, internal) + classes_loaded[name] = cls - for k, v in loads(external).items(): + external = loads(external) + for k, v in external.items(): if isinstance(k, tuple): t, k = k if t == 'property': @@ -215,105 +416,129 @@ def load_local_class(bytes): v = property(fget, fset, fdel, doc) if t == 'staticmethod': + v = load_closure(v) v = staticmethod(v) + if t == 'classmethod': + v = load_closure(v) + v = classmethod(v) + + if t == 'method': + v = load_closure(v) + setattr(cls, k, v) return cls + def reduce_class(obj): name = obj.__name__ module = getattr(obj, "__module__", None) - if module == '__main__' and name not in ('MyPickler', 'RecursiveFunctionPlaceholder'): + if module == '__main__' and name not in ( + 'MyPickler', 'RecursiveFunctionPlaceholder'): result = load_local_class, (dump_local_class(obj),) return result return name -CLS_TYPES = [types.TypeType, types.ClassType] + def dump_method(method): - obj = method.im_self - cls = method.im_class - func = method.im_func - if cls in CLS_TYPES: - cls_name = CLS_TYPES.index(cls) - else: - cls_name = cls.__name__ + obj = method.__self__ or method.__self__.__class__ + func = method.__func__ + + return dumps((obj, func.__name__)) - return dumps((obj, cls_name, func)) def load_method(bytes): - obj, cls_name, func = loads(bytes) # cls referred in func.func_globals - if isinstance(cls_name, int): - cls = CLS_TYPES[cls_name] - else: - cls = classes_loaded[cls_name] + _self, func_name = loads(bytes) + return getattr(_self, func_name) - return types.MethodType(func, obj, cls) def reduce_method(method): - module = method.im_func.__module__ - return load_method, (dump_method(method), ) + module = method.__func__.__module__ + return load_method, (dump_method(method),) + MyPickler.register(types.LambdaType, reduce_function) -MyPickler.register(types.ClassType, reduce_class) -MyPickler.register(types.TypeType, reduce_class) +if six.PY2: + MyPickler.register(types.ClassType, reduce_class) + +MyPickler.register(type, reduce_class) MyPickler.register(types.MethodType, reduce_method) if __name__ == "__main__": assert marshalable(None) assert marshalable("") assert marshalable(u"") - assert not marshalable(buffer("")) + assert not marshalable(memoryview(b"")) + assert marshalable(0) assert marshalable(0) - assert marshalable(0L) assert marshalable(0.0) assert marshalable(True) - assert marshalable(complex(1,1)) - assert marshalable((1,1)) - assert marshalable([1,1]) - assert marshalable(set([1,1])) - assert marshalable({1:None}) + assert marshalable(complex(1, 1)) + assert marshalable((1, 1)) + assert marshalable([1, 1]) + assert marshalable(set([1, 1])) + assert marshalable({1: None}) some_global = 'some global' + + def glob_func(s): return "glob:" + s + + def get_closure(x): glob_func(some_global) last = " last" + def foo(y): return "foo: " + y + def the_closure(a, b=1): marshal.dumps(a) - return (a * x + int(b), glob_func(foo(some_global)+last)) + return (a * x + int(b), glob_func(foo(some_global) + last)) + return the_closure + f = get_closure(10) ff = loads(dumps(f)) - #print globals() - print f(2) - print ff(2) + # print globals() + print(f(2)) + print(ff(2)) glob_func = loads(dumps(glob_func)) get_closure = loads(dumps(get_closure)) + # Test recursive functions - def fib(n): return n if n <= 1 else fib(n-1) + fib(n-2) + def fib(n): + return n if n <= 1 else fib(n - 1) + fib(n - 2) + + assert fib(8) == loads(dumps(fib))(8) + class Foo1: + def foo(self): return 1234 + class Foo2(object): + def foo(self): return 5678 + class Foo3(Foo2): x = 1111 def foo(self): return super(Foo3, self).foo() + Foo3.x + class Foo4(object): + @classmethod def x(cls): return 1 @@ -326,6 +551,13 @@ def y(self): def z(): return 3 + def recursive(self, x): + if x <= 0: + return x + else: + return self.recursive(x - 1) + + df1 = dumps(Foo1) df2 = dumps(Foo2) df3 = dumps(Foo3) @@ -351,6 +583,10 @@ def z(): assert f3.foo() == 5678 + 1111 assert Foo4.x() == 1 - f = loads(dumps(lambda:(some_global for i in xrange(1)))) - print list(f()) + recursive = Foo4().recursive + _recursive = loads(dumps(recursive)) + assert _recursive(5) == 0 + + f = loads(dumps(lambda: (some_global for i in range(1)))) + print(list(f())) assert list(f()) == [some_global] diff --git a/dpark/shuffle.py b/dpark/shuffle.py index 1fd537d8..984d2e10 100644 --- a/dpark/shuffle.py +++ b/dpark/shuffle.py @@ -1,234 +1,453 @@ -import os, os.path +from __future__ import absolute_import +from __future__ import print_function +import os +import os.path import random -import urllib -import logging +import six +from six.moves import urllib, queue, range, zip, reduce, cPickle as pickle import marshal import struct import time -import cPickle -import gzip -import Queue import heapq -import platform - -from dpark.util import decompress, spawn +import itertools +from operator import itemgetter +from itertools import islice +from functools import wraps + +try: + import cStringIO as StringIO +except ImportError: + from six import BytesIO as StringIO + +import dpark.conf +from dpark.utils import compress, decompress, spawn, atomic_file +from dpark.utils.memory import ERROR_TASK_OOM +from dpark.utils.log import get_logger from dpark.env import env from dpark.tracker import GetValueMessage, SetValueMessage +from dpark.utils.heaponkey import HeapOnKey +from dpark.dependency import AggregatorBase +from dpark.utils.nested_groupby import GroupByNestedIter, cogroup_no_dup -MAX_SHUFFLE_MEMORY = 2000 # 2 GB +logger = get_logger(__name__) -logger = logging.getLogger(__name__) +# readable +F_MAPPING = { + (True, True): b'M', + (False, True): b'P', + (True, False): b'm', + (False, False): b'p' +} -class LocalFileShuffle: - serverUri = None - shuffleDir = None - @classmethod - def initialize(cls, isMaster): - cls.shuffleDir = [p for p in env.get('WORKDIR') - if os.path.exists(os.path.dirname(p))] - if not cls.shuffleDir: - return - cls.serverUri = env.get('SERVER_URI') - logger.debug("shuffle dir: %s", cls.shuffleDir) +F_MAPPING_R = dict([(v, k) for k, v in F_MAPPING.items()]) - @classmethod - def getOutputFile(cls, shuffleId, inputId, outputId, datasize=0): - path = os.path.join(cls.shuffleDir[0], str(shuffleId), str(inputId)) - if not os.path.exists(path): - try: os.makedirs(path) - except OSError: pass - p = os.path.join(path, str(outputId)) - - if datasize > 0 and len(cls.shuffleDir) > 1: - st = os.statvfs(path) - free = st.f_bfree * st.f_bsize - ratio = st.f_bfree * 1.0 / st.f_blocks - if free < max(datasize, 1<<30) or ratio < 0.66: - d2 = os.path.join(random.choice(cls.shuffleDir[1:]), str(shuffleId), str(inputId)) - if not os.path.exists(d2): - try: os.makedirs(d2) - except IOError: pass - assert os.path.exists(d2), 'create %s failed' % d2 - p2 = os.path.join(d2, str(outputId)) - os.symlink(p2, p) - if os.path.islink(p2): - os.unlink(p2) # p == p2 - return p2 - return p - @classmethod - def getServerUri(cls): - return cls.serverUri +def pack_header(length, is_marshal, is_sorted): + flag = F_MAPPING[(is_marshal, is_sorted)] + return flag + struct.pack("I", length) -class ShuffleFetcher: - def fetch(self, shuffleId, reduceId, func): - raise NotImplementedError - def stop(self): - pass +def unpack_header(head): + l = len(head) + if l != 5: + raise IOError("fetch bad head length %d" % (l,)) + flag = head[:1] + is_marshal, is_sorted = F_MAPPING_R[flag] + length, = struct.unpack("I", head[1:5]) + return length, is_marshal, is_sorted -class SimpleShuffleFetcher(ShuffleFetcher): - def fetch_one(self, uri, shuffleId, part, reduceId): - if uri == LocalFileShuffle.getServerUri(): - # urllib can open local file - url = LocalFileShuffle.getOutputFile(shuffleId, part, reduceId) + +def write_buf(stream, buf, is_marshal): + buf = compress(buf) + size = len(buf) + stream.write(pack_header(size, is_marshal, True)) + stream.write(buf) + return size + 4 + + +class AutoBatchedSerializer(object): + """ + Choose the size of batch automatically based on the size of object + """ + + size_loaded = 0 + + def __init__(self, best_size=1 << 17): + self.best_size = best_size + self.max_num = 0 + self.max_size = 0 + self.use_marshal = True + self.num_batch = 0 + self.file_size = 0 + + def load_stream(self, stream): + while True: + head = stream.read(5) + if not head: + return + length, is_marshal, is_sorted = unpack_header(head) + assert (is_sorted) + buf = stream.read(length) + if len(buf) < length: + raise IOError("length not match: expected %d, but got %d" % (length, len(buf))) + + buf = decompress(buf) + AutoBatchedSerializer.size_loaded += len(buf) + if is_marshal: + vs = marshal.loads(buf) + else: + vs = pickle.loads(buf) + for v in vs: + yield v + + def dump_stream(self, iterator, stream): + self._dump_stream(iter(iterator), stream) + logger.debug("max batch num = %d, max batch size = %d", self.max_num, self.max_size) + + def _dump_stream(self, iterator, stream): + batch_num = 1 + + while True: + vs = list(itertools.islice(iterator, batch_num)) + self.num_batch += 1 + if not vs: + break + batch_num = self._dump_batch(stream, vs, batch_num) + + def _dump_batch(self, stream, vs, batch_num): + if self.use_marshal: + try: + buf = marshal.dumps(vs) + except: + buf = pickle.dumps(vs, -1) + self.use_marshal = False + else: + buf = pickle.dumps(vs, -1) + + mem_size = len(buf) + self.file_size += write_buf(stream, buf, self.use_marshal) + + if mem_size < self.best_size: + batch_num *= 2 + if batch_num > self.max_num: + self.max_num = batch_num else: - url = "%s/%d/%d/%d" % (uri, shuffleId, part, reduceId) - logger.debug("fetch %s", url) + if mem_size > self.best_size * 2 and batch_num > 1: + batch_num //= 2 + if mem_size > self.max_size: + self.max_size = mem_size + return batch_num - tries = 2 + +class GroupByAutoBatchedSerializer(AutoBatchedSerializer): + + def _dump_stream(self, iterator, stream): + batch_num = 1 + + def _batching(): + batch = [] + num = 0 + + for k, vs in iterator: + n = len(vs) + if n + num <= batch_num: + batch.append((k, vs)) + num += n + else: + if batch: + yield batch + batch = [] + num = 0 + if n >= batch_num: + sub_it = iter(vs) + while True: + sub_vs = list(itertools.islice(sub_it, batch_num)) + if not sub_vs: + break + yield [(k, sub_vs)] + else: + batch.append((k, vs)) + num = n + if batch: + yield batch + + for k_vs in _batching(): + self.num_batch += 1 + batch_num = self._dump_batch(stream, k_vs, batch_num) + + +def get_serializer(rddconf): + if rddconf.iter_group and (rddconf.is_groupby or rddconf.is_cogroup): + return GroupByAutoBatchedSerializer() + else: + return AutoBatchedSerializer() + + +def fetch_with_retry(f): + MAX_RETRY = 3 + RETRY_INTERVALS = [1, 10] + + @wraps(f) + def _(self): + self.num_batch_done = 0 while True: try: - f = urllib.urlopen(url) - if f.code == 404: - f.close() - raise IOError("not found") - - d = f.read() - flag = d[:1] - length, = struct.unpack("I", d[1:5]) + for items in islice(f(self), self.num_batch_done, None): + self.num_batch_done += 1 + yield items + if self.num_retry > 0: + logger.info("Fetch retry %d success for url %s, num_batch %d ", self.num_retry, self.url, + self.num_batch_done) + break + except Exception as e: + self.num_retry += 1 + msg = "Fetch failed for url %s, tried %d/%d times. Exception: %s. " % ( + self.url, self.num_retry, MAX_RETRY, e) + fail_fast = False + emsg = str(e) + if any([emsg.find(s) >= 0 for s in ["404"]]): + # "many open file", + fail_fast = True + msg += "no need to retry." + if fail_fast or self.num_retry >= MAX_RETRY: + logger.warning(msg) + from dpark.task import FetchFailed + raise FetchFailed(self.uri, self.sid, self.mid, self.rid) + else: + sleep_time = RETRY_INTERVALS[self.num_retry - 1] + msg += "sleep %d secs" % (sleep_time,) + logger.debug(msg) + time.sleep(sleep_time) + + return _ + + +class RemoteFile(object): + num_open = 0 + + def __init__(self, uri, shuffle_id, map_id, reduce_id): + self.uri = uri + self.sid = shuffle_id + self.mid = map_id + self.rid = reduce_id + self.url = ShuffleWorkDir(shuffle_id, map_id, reduce_id).restore(uri) + logger.debug("fetch %s", self.url) + + self.num_retry = 0 + self.num_batch_done = 0 + + def open(self): + f = urllib.request.urlopen(self.url) + if f.code == 404: + f.close() + raise IOError("not found") + exp_size = int(f.headers['content-length']) + return f, exp_size + + @fetch_with_retry + def unsorted_batches(self): + f = None + # TEST_RETRY = True + try: + f, exp_size = self.open() + total_size = 0 + + while True: + head = f.read(5) + if len(head) == 0: + break + length, is_marshal, is_sorted = unpack_header(head) + assert (not is_sorted) + total_size += length + 5 + d = f.read(length) if length != len(d): - raise ValueError("length not match: expected %d, but got %d" % (length, len(d))) - d = decompress(d[5:]) - f.close() - if flag == 'm': - d = marshal.loads(d) - elif flag == 'p': - d = cPickle.loads(d) + raise IOError( + "length not match: expected %d, but got %d" % + (length, len(d))) + d = decompress(d) + if is_marshal: + items = marshal.loads(d) else: - raise ValueError("invalid flag") - return d - except Exception, e: - logger.debug("Fetch failed for shuffle %d, reduce %d, %d, %s, %s, try again", - shuffleId, reduceId, part, url, e) - tries -= 1 - if not tries: - logger.warning("Fetch failed for shuffle %d, reduce %d, %d, %s, %s", - shuffleId, reduceId, part, url, e) - from dpark.schedule import FetchFailed - raise FetchFailed(uri, shuffleId, part, reduceId) - time.sleep(2**(2-tries)*0.1) - - def fetch(self, shuffleId, reduceId, func): - logger.debug("Fetching outputs for shuffle %d, reduce %d", shuffleId, reduceId) - serverUris = env.mapOutputTracker.getServerUris(shuffleId) - parts = zip(range(len(serverUris)), serverUris) - random.shuffle(parts) - for part, uri in parts: - d = self.fetch_one(uri, shuffleId, part, reduceId) - func(d.iteritems()) + try: + items = pickle.loads(d) + except: + time.sleep(1) + items = pickle.loads(d) + yield items + + # if TEST_RETRY and self.num_retry == 0: + # raise Exception("test_retry") + + if total_size != exp_size: + raise IOError( + "fetch size not match: expected %d, but got %d" % + (exp_size, total_size)) + + env.task_stats.bytes_fetch += exp_size + finally: + if f: + f.close() + + @fetch_with_retry + def sorted_items(self): + f = None + try: + serializer = AutoBatchedSerializer() + self.num_open += 1 + f, exp_size = self.open() + for obj in serializer.load_stream(f): + yield obj + env.task_stats.bytes_fetch += exp_size + finally: + # rely on GC to close if generator not exhausted + # so Fetcher must not be an attr of RDD + if f: + f.close() + self.num_open -= 1 + + +class ShuffleFetcher(object): + + @classmethod + def _get_uris(cls, shuffle_id): + uris = MapOutputTracker.get_locs(shuffle_id) + mapid_uris = list(zip(list(range(len(uris))), uris)) + random.shuffle(mapid_uris) + return mapid_uris + + @classmethod + def get_remote_files(cls, shuffle_id, reduce_id): + uris = cls._get_uris(shuffle_id) + return [RemoteFile(uri, shuffle_id, map_id, reduce_id) for map_id, uri in uris] + + def fetch(self, shuffle_id, reduce_id, merge_func): + raise NotImplementedError + + def stop(self): + pass + + +class SimpleShuffleFetcher(ShuffleFetcher): + + def fetch(self, shuffle_id, reduce_id, merge_func): + logger.debug( + "Fetching outputs for shuffle %d, reduce %d", + shuffle_id, reduce_id) + for f in self.get_remote_files(): + for items in f.unsorted_batches(): + merge_func(items) class ParallelShuffleFetcher(SimpleShuffleFetcher): + def __init__(self, nthreads): self.nthreads = nthreads - self.start() + self._started = False def start(self): - self.requests = Queue.Queue() - self.results = Queue.Queue(self.nthreads) - self.threads = [spawn(self._worker_thread) for i in range(self.nthreads)] + if self._started: + return + + self._started = True + self.requests = queue.Queue() + self.results = queue.Queue(self.nthreads) + self.threads = [spawn(self._fetch_thread) + for i in range(self.nthreads)] + + def _fetch_thread(self): + from dpark.task import FetchFailed - def _worker_thread(self): - from dpark.schedule import FetchFailed while True: - r = self.requests.get() - if r is None: + f = self.requests.get() + if f is None: break - - uri, shuffleId, part, reduceId = r try: - d = self.fetch_one(*r) - self.results.put((shuffleId, reduceId, part, d)) - except FetchFailed, e: + for items in f.unsorted_batches(): + if not self._started: + break + self.results.put((items, f.mid)) + if not self._started: + break + self.results.put(1) + except FetchFailed as e: + if not self._started: + break self.results.put(e) + break - def fetch(self, shuffleId, reduceId, func): - logger.debug("Fetching outputs for shuffle %d, reduce %d", shuffleId, reduceId) - serverUris = env.mapOutputTracker.getServerUris(shuffleId) - if not serverUris: - return - - parts = zip(range(len(serverUris)), serverUris) - random.shuffle(parts) - for part, uri in parts: - self.requests.put((uri, shuffleId, part, reduceId)) - - from dpark.schedule import FetchFailed - for i in xrange(len(serverUris)): + def fetch(self, shuffle_id, reduce_id, merge_func): + self.start() + files = self.get_remote_files(shuffle_id, reduce_id) + for f in files: + self.requests.put(f) + + t = time.time() + from dpark.task import FetchFailed + num_done = 0 + while num_done < len(files): r = self.results.get() - if isinstance(r, FetchFailed): - self.stop() # restart - self.start() + if r == 1: + num_done += 1 + elif isinstance(r, FetchFailed): + self.stop() raise r + else: + items, map_id = r + merge_func(items, map_id) - sid, rid, part, d = r - func(d.iteritems()) + env.task_stats.secs_fetch = time.time() - t def stop(self): + if not self._started: + return + logger.debug("stop parallel shuffle fetcher ...") + self._started = False while not self.requests.empty(): self.requests.get_nowait() - while not self.results.empty(): - self.results.get_nowait() for i in range(self.nthreads): self.requests.put(None) - for t in self.threads: - t.join() - -class Merger(object): - - def __init__(self, total, mergeCombiner): - self.mergeCombiner = mergeCombiner - self.combined = {} - - def merge(self, items): - combined = self.combined - mergeCombiner = self.mergeCombiner - for k,v in items: - o = combined.get(k) - combined[k] = mergeCombiner(o, v) if o is not None else v - - def __iter__(self): - return self.combined.iteritems() + N = 5 + for _ in range(N): + while not self.results.empty(): + self.results.get_nowait() + for t in self.threads: + t.join(1) + if all([not t.isAlive() for t in self.threads]): + return + else: + logger.info("FIXME: fail to join fetcher threads") -class CoGroupMerger(object): - def __init__(self, size): - self.size = size - self.combined = {} - def get_seq(self, k): - return self.combined.setdefault(k, tuple([[] for i in range(self.size)])) +class SortShuffleFetcher(ShuffleFetcher): - def append(self, i, items): - for k, v in items: - self.get_seq(k)[i].append(v) + def get_iters(self, shuffle_id, reduce_id): + return [f.sorted_items() for f in self.get_remote_files(shuffle_id, reduce_id)] - def extend(self, i, items): - for k, v in items: - self.get_seq(k)[i].extend(v) + def fetch(self, shuffle_id, reduce_id, merge_func): + merge_func(self.get_iters(shuffle_id, reduce_id)) - def __iter__(self): - return self.combined.iteritems() def heap_merged(items_lists, combiner): heap = [] - def pushback(it): + + def pushback(_it, _i): try: - k,v = it.next() + _k, _v = next(_it) # put i before value, so do not compare the value - heapq.heappush(heap, (k, i, v)) + heapq.heappush(heap, (_k, i, _v)) except StopIteration: pass + for i, it in enumerate(items_lists): - if isinstance(it, list): - items_lists[i] = it = (k for k in it) - pushback(it) - if not heap: return + pushback(it, i) + + if not heap: + return last_key, i, last_value = heapq.heappop(heap) - pushback(items_lists[i]) + pushback(items_lists[i], i) while heap: k, i, v = heapq.heappop(heap) @@ -237,166 +456,395 @@ def pushback(it): last_key, last_value = k, v else: last_value = combiner(last_value, v) - pushback(items_lists[i]) + pushback(items_lists[i], i) yield last_key, last_value -class sorted_items(object): - next_id = 0 - @classmethod - def new_id(cls): - cls.next_id += 1 - return cls.next_id - def __init__(self, items): - self.id = self.new_id() - self.path = path = os.path.join(LocalFileShuffle.shuffleDir, - 'shuffle-%d-%d.tmp.gz' % (os.getpid(), self.id)) - f = gzip.open(path, 'wb+') +class SortedItemsOnDisk(object): - items = sorted(items) - try: - for i in items: - s = marshal.dumps(i) - f.write(struct.pack("I", len(s))) - f.write(s) - self.loads = marshal.loads - except Exception, e: - f.rewind() - for i in items: - s = cPickle.dumps(i) - f.write(struct.pack("I", len(s))) - f.write(s) - self.loads = cPickle.loads - f.close() - - self.f = gzip.open(path) - self.c = 0 + def __init__(self, items, rddconf): + self.path = path = env.workdir.alloc_tmp("sorted_items") + with atomic_file(path, bufsize=4096) as f: + if not isinstance(items, list): + items = list(items) + items.sort(key=itemgetter(0)) + serializer = get_serializer(rddconf) + serializer.dump_stream(items, f) + self.size = f.tell() + self.num_batch = serializer.num_batch def __iter__(self): - self.f = gzip.open(self.path) - self.c = 0 - return self - - def next(self): - f = self.f - b = f.read(4) - if not b: - f.close() + serializer = AutoBatchedSerializer() + with open(self.path, 'rb') as f: + for obj in serializer.load_stream(f): + yield obj + + def __dealloc__(self): + try: if os.path.exists(self.path): os.remove(self.path) - raise StopIteration - sz, = struct.unpack("I", b) - self.c += 1 - return self.loads(f.read(sz)) + except Exception: + pass - def __dealloc__(self): - self.f.close() - if os.path.exists(self.path): - os.remove(self.path) +class Merger(object): + + def __init__(self, rddconf, aggregator=None, size=None, api_callsite=None): + self.rddconf = rddconf + self.aggregator = aggregator + self.size = size + self.api_callsite = api_callsite + + @classmethod + def get(cls, rddconf, aggregator=None, size=0, api_callsite=None): + if rddconf.sort_merge: + # all mergers keep order + c = SortMerger + if rddconf.is_cogroup: + if rddconf.iter_group: + c = IterCoGroupSortMerger + elif rddconf.is_groupby: + if rddconf.iter_group: + c = IterGroupBySortMerger + else: + c = DiskHashMerger + if rddconf.is_groupby: + if rddconf.ordered_group: + c = OrderedGroupByDiskHashMerger + elif rddconf.is_cogroup: + if rddconf.ordered_group: + c = OrderedCoGroupDiskHashMerger + else: + c = CoGroupDiskHashMerger + logger.debug("%s %s", c, rddconf) + return c(rddconf, aggregator, size, api_callsite) + + +class DiskHashMerger(Merger): + + def __init__(self, rddconf, aggregator=None, size=None, api_callsite=None): + Merger.__init__(self, rddconf, aggregator, size, api_callsite) -class DiskMerger(Merger): - def __init__(self, total, combiner): - Merger.__init__(self, total, combiner) - self.total = total - self.archives = [] - self.base_memory = self.get_used_memory() - self.max_merge = None - self.merged = 0 - - def get_used_memory(self): - if platform.system() == 'Linux': - for line in open('/proc/self/status'): - if line.startswith('VmRSS:'): - return int(line.split()[1]) >> 10 - return 0 - - def merge(self, items): - Merger.merge(self, items) - - self.merged += 1 - #print 'used', self.merged, self.total, self.get_used_memory() - self.base_memory, self.base_memory - if self.max_merge is None: - if self.merged < self.total/5 and self.get_used_memory() - self.base_memory > MAX_SHUFFLE_MEMORY: - self.max_merge = self.merged - - if self.max_merge is not None and self.merged >= self.max_merge: - t = time.time() - self.rotate() - self.merged = 0 - #print 'after rotate', self.get_used_memory() - self.base_memory, time.time() - t - - def rotate(self): - self.archives.append(sorted_items(self.combined.iteritems())) self.combined = {} + self.use_disk = rddconf.disk_merge + if self.use_disk: + env.meminfo.ratio = rddconf.dump_mem_ratio + + self.archives = [] + + self.rotate_time = 0 + self.last_rotate_ts = time.time() + self.rotate_num = 0 + self.total_size = 0 + + def _rotate(self): + total_size = self.total_size + t0 = time.time() + time_since_last = t0 - self.last_rotate_ts + dict_size = len(self.combined) + rss_before = env.meminfo.rss_rt + size = self._dump() + self.total_size += size + rss_after = env.meminfo.rss_rt + t1 = time.time() + rotate_time = t1 - t0 + self.last_rotate_ts = t1 + self.rotate_time += rotate_time + self.rotate_num += 1 + max_rotate = 1000 + if self.rotate_num > max_rotate: + logger.warnging('more than %d rotation. exit!', max_rotate) + os._exit(ERROR_TASK_OOM) + + env.meminfo.after_rotate() + + _log = logger.info if dpark.conf.LOG_ROTATE else logger.debug + _log('rotate %d: use %.2f sec, since last %.2f secs, dict_size 0x%x,' + 'mem %d -> %d MB, disk size +%d = %d MB', + self.rotate_num, rotate_time, time_since_last, dict_size, + rss_before >> 20, rss_after >> 20, size >> 20, total_size >> 20) + + return env.meminfo.mem_limit_soft + + def disk_size(self): + return sum([a.size for a in self.archives]) + + def _dump(self): + import gc + items = self.combined.items() + f = SortedItemsOnDisk(items, self.rddconf) + self.archives.append(f) + del items + + if self.rddconf.is_groupby: + for v in self.combined.itervalues(): + del v[:] + self.combined.clear() + gc.collect() + return f.size + + def merge(self, items, map_id, dep_id=0): + mem_limit = env.meminfo.mem_limit_soft + use_disk = self.use_disk + try: + env.meminfo.check = use_disk + self._merge(items, map_id, dep_id, use_disk, env.meminfo, mem_limit) + finally: + env.meminfo.check = True + + def _get_merge_function(self): + return self.aggregator.mergeCombiners + + def _merge(self, items, map_id, dep_id, use_disk, meminfo, mem_limit): + combined = self.combined + merge_combiner = self.aggregator.mergeCombiners + for k, v in items: + o = combined.get(k) + combined[k] = merge_combiner(o, v) if o is not None else v + + if use_disk and meminfo.rss > mem_limit: + mem_limit = self._rotate() + def __iter__(self): if not self.archives: - return self.combined.iteritems() + return six.iteritems(self.combined) + items = self.combined.items() + items.sort(key=itemgetter(0)) + combined = items + self.archives.append(iter(combined)) + iters = list(map(iter, self.archives)) + if self.rddconf.is_groupby and self.rddconf.iter_group: + heap = HeapOnKey(key=lambda x: x[0], min_heap=True) + it = GroupByNestedIter(heap.merge(iters), "") + else: + it = heap_merged(iters, self._get_merge_function()) + return it - if self.combined: - self.rotate() - return heap_merged(self.archives, self.mergeCombiner) -class BaseMapOutputTracker(object): - def registerMapOutputs(self, shuffleId, locs): - pass +class OrderedGroupByDiskHashMerger(DiskHashMerger): - def getServerUris(self): - pass + def _merge(self, items, map_id, dep_id, use_disk, meminfo, mem_limit): + combined = self.combined + for k, v in items: + o = combined.get(k) + iv = (map_id, v) + if o is None: + combined[k] = [iv] + else: + o.append(iv) + if use_disk and meminfo.rss > mem_limit: + mem_limit = self._rotate() - def stop(self): - pass + def __iter__(self): + it = DiskHashMerger.__iter__(self) + merge_combiner = self.aggregator.mergeCombiners + for k, ivs in it: + ivs.sort(key=itemgetter(0)) + cb = reduce(merge_combiner, (v for _, v in ivs)) + yield k, cb + + +class CoGroupDiskHashMerger(DiskHashMerger): + + def __init__(self, rddconf, aggregator=None, size=None, api_callsite=None): + DiskHashMerger.__init__(self, rddconf, aggregator, size, api_callsite) + self.direct_upstreams = [] + + def _get_merge_function(self): + def _merge(x, y): + for i in range(self.size): + x[i].extend(y[i]) + return x + + return _merge + + def _merge(self, items, map_id, dep_id, use_disk, meminfo, mem_limit): + combined = self.combined + if map_id < 0: + for k, v in items: + t = combined.get(k) + if t is None: + combined[k] = t = tuple([[] for _ in range(self.size)]) + t[dep_id].append(v) + if use_disk and meminfo.rss > mem_limit: + mem_limit = self._rotate() + else: + for k, vs in items: + t = combined.get(k) + if t is None: + combined[k] = t = tuple([[] for _ in range(self.size)]) + t[dep_id].extend(vs) + + +class OrderedCoGroupDiskHashMerger(CoGroupDiskHashMerger): + + def _merge(self, items, map_id, dep_id, use_disk, meminfo, mem_limit): + combined = self.combined + if map_id < 0: + self.upstreams.append(dep_id) + for k, v in items: + t = combined.get(k) + if t is None: + combined[k] = t = tuple([[] for _ in range(self.size)]) + t[dep_id].append(v) + if use_disk and meminfo.rss > mem_limit: + mem_limit = self._rotate() + else: + for k, vs in items: + t = combined.get(k) + if t is None: + combined[k] = t = tuple([[] for _ in range(self.size)]) + t[dep_id].append((map_id, vs)) + if use_disk and meminfo.rss > mem_limit: + mem_limit = self._rotate() + + def __iter__(self): + it = DiskHashMerger.__iter__(self) + + direct_upstreams = self.direct_upstreams + for k, groups in it: + t = list([[] for _ in range(self.size)]) + for i, g in enumerate(groups): + if g: + if i in direct_upstreams: + t[i] = g + else: + g.sort(key=itemgetter(0)) + g1 = [] + for _, vs in g: + g1.extend(vs) + t[i] = g1 + yield k, tuple(t) -class MapOutputTracker(BaseMapOutputTracker): - def __init__(self): - self.client = env.trackerClient - logger.debug("MapOutputTracker started") - - def registerMapOutputs(self, shuffleId, locs): - self.client.call(SetValueMessage('shuffle:%s' % shuffleId, locs)) - - def getServerUris(self, shuffleId): - locs = self.client.call(GetValueMessage('shuffle:%s' % shuffleId)) - logger.debug("Fetch done: %s", locs) - return locs - -def test(): - l = [] - for i in range(10): - d = zip(range(10000), range(10000)) - l.append(sorted_items(d)) - hl = heap_merged(l, lambda x,y:x+y) - for i in range(10): - print i, hl.next() - - import logging - logging.basicConfig(level=logging.INFO) - from dpark.env import env - import cPickle - env.start(True) - - path = LocalFileShuffle.getOutputFile(1, 0, 0) - f = open(path, 'w') - f.write(cPickle.dumps([('key','value')], -1)) - f.close() - - uri = LocalFileShuffle.getServerUri() - env.mapOutputTracker.registerMapOutputs(1, [uri]) - fetcher = SimpleShuffleFetcher() - def func(k,v): - assert k=='key' - assert v=='value' - fetcher.fetch(1, 0, func) - - tracker = MapOutputTracker(True) - tracker.registerMapOutputs(2, [None, uri, None, None, None]) - assert tracker.getServerUris(2) == [None, uri, None, None, None] - ntracker = MapOutputTracker(False) - assert ntracker.getServerUris(2) == [None, uri, None, None, None] - ntracker.stop() - tracker.stop() - -if __name__ == '__main__': - test() + +class SortMergeAggregator(AggregatorBase): + + def __init__(self, mergeCombiners): + # each item is a combiner + self.mergeValue = mergeCombiners + + def createCombiner(self, v): + return v + + +class CoGroupSortMergeAggregator(AggregatorBase): + def __init__(self, size): + self.size = size + + def createCombiner(self, v): + # v = (rdd_index, value) + values = tuple([[] for _ in range(self.size)]) + values[v[0]].extend(v[1]) + return values + + def mergeValue(self, c, v): + c[v[0]].extend(v[1]) + return c + + +class SortMerger(Merger): + + def __init__(self, rddconf, aggregator=None, size=None, api_callsite=None): + Merger.__init__(self, rddconf, aggregator, size, api_callsite) + + if aggregator: + self.aggregator = SortMergeAggregator(self.aggregator.mergeCombiners) + else: + self.aggregator = CoGroupSortMergeAggregator(size) + self.combined = iter([]) + + self.paths = [] + + def _merge_sorted(self, iters): + heap = HeapOnKey(key=lambda x: x[0], min_heap=True) + merged = heap.merge(iters) + return self.aggregator.aggregate_sorted(merged) + + def _disk_merge_sorted(self, iters): + t = time.time() + s = AutoBatchedSerializer() + iters = iter(iters) + while True: + batch = list(islice(iters, 100)) + if not batch: + break + path = env.workdir.alloc_tmp_file("sort_merger") + with open(path, 'wb') as f: + s.dump_stream(self._merge_sorted(batch), f) + self.paths.append(path) + env.task_stats.num_fetch_rotate += 1 + + files = [s.load_stream(open(p)) for p in self.paths] + env.task_stats.secs_fetch = time.time() - t + return self._merge_sorted(files) + + def merge(self, iters): + if self.rddconf.disk_merge or len(iters) > dpark.conf.MAX_OPEN_FILE: + merged = self._disk_merge_sorted(iters) + else: + merged = self._merge_sorted(iters) + + self.combined = merged + + def __iter__(self): + return self.combined + + +class IterGroupBySortMerger(SortMerger): + + def _merge_sorted(self, iters): + heap = HeapOnKey(key=lambda x: x[0], min_heap=True) + return GroupByNestedIter(heap.merge(iters), self.api_callsite) + + +class IterCoGroupSortMerger(SortMerger): + + def _merge_sorted(self, iters): + # each item like + return cogroup_no_dup(list(map(iter, iters))) + + +class MapOutputTracker(object): + + @classmethod + def get_key(cls, shuffle_id): + return 'shuffle:{}'.format(shuffle_id) + + @classmethod + def set_locs(cls, shuffle_id, locs): + key = cls.get_key(shuffle_id) + env.trackerServer.set(key, locs) + + @classmethod + def get_locs(cls, shuffle_id): + key = cls.get_key(shuffle_id) + if env.trackerServer: + return env.trackerServer.get(key) + else: + return env.trackerClient.call(GetValueMessage(key)) + + +class ShuffleWorkDir(object): + + def __init__(self, shuffle_id, input_id, output_id): + self.subpath = os.path.join(str(shuffle_id), str(input_id), str(output_id)) + + def get(self): + return env.workdir.get_path(self.subpath) + + @classmethod + def alloc_tmp(cls, mem_first=True, datasize=0): + return env.workdir.alloc_tmp_file("shuffle", mem_first, datasize) + + def export(self, tmppath): + return env.workdir.export(tmppath, self.subpath) + + def restore(self, uri): + if uri == env.server_uri: + # urllib can open local file + url = 'file://' + self.get() + else: + url = "%s/%s" % (uri, self.subpath) + return url diff --git a/dpark/table.py b/dpark/table.py index 5a475956..09065c81 100644 --- a/dpark/table.py +++ b/dpark/table.py @@ -1,6 +1,6 @@ +from __future__ import absolute_import import os, sys import re -import logging from collections import namedtuple import itertools @@ -8,52 +8,55 @@ from dpark.rdd import DerivedRDD, OutputTableFileRDD from dpark.dependency import Aggregator, OneToOneDependency +from six.moves import range +from six.moves import zip +from functools import reduce try: from pyhll import HyperLogLog except ImportError: - from dpark.hyperloglog import HyperLogLog + from dpark.utils.hyperloglog import HyperLogLog -from hotcounter import HotCounter +from dpark.utils.hotcounter import HotCounter SimpleAggs = { - 'sum': lambda x,y: x+y, - 'last': lambda x,y: y, - 'min': lambda x,y: x if x < y else y, - 'max': lambda x,y: x if x > y else y, + 'sum': lambda x, y: x + y, + 'last': lambda x, y: y, + 'min': lambda x, y: x if x < y else y, + 'max': lambda x, y: x if x > y else y, } FullAggs = { - 'avg': ( - lambda v:(v, 1), - lambda (s,c), v: (s+v, c+1), - lambda (s1,c1), (s2,c2):(s1+s2, c1+c2), - lambda (s,c): float(s)/c, - ), - 'count': ( - lambda v: 1 if v is not None else 0, - lambda s,v: s + (1 if v is not None else 0), - lambda s1,s2: s1+s2, - lambda s: s - ), - 'adcount': ( - lambda v: HyperLogLog([v]), - lambda s,v: s.add(v) or s, - lambda s1,s2: s1.update(s2) or s1, - lambda s: len(s) - ), - 'group_concat': ( - lambda v: [v], - lambda s,v: s.append(v) or s, - lambda s1,s2: s1.extend(s2) or s1, - lambda s: ','.join(s), - ), - 'top': ( - lambda v: HotCounter([v], 20), - lambda s,v: s.add(v) or s, - lambda s1,s2: s1.update(s2) or s1, - lambda s: s.top(20) - ), + 'avg': ( + lambda v: (v, 1), + lambda s_c, v: (s_c[0] + v, s_c[1] + 1), + lambda x, y: (x[0] + y[0], x[1] + y[1]), + lambda s_c: float(s_c[0]) / s_c[1], + ), + 'count': ( + lambda v: 1 if v is not None else 0, + lambda s, v: s + (1 if v is not None else 0), + lambda s1, s2: s1 + s2, + lambda s: s + ), + 'adcount': ( + lambda v: HyperLogLog([v]), + lambda s, v: s.add(v) or s, + lambda s1, s2: s1.update(s2) or s1, + lambda s: len(s) + ), + 'group_concat': ( + lambda v: [v], + lambda s, v: s.append(v) or s, + lambda s1, s2: s1.extend(s2) or s1, + lambda s: ','.join(s), + ), + 'top': ( + lambda v: HotCounter([v], 20), + lambda s, v: s.add(v) or s, + lambda s1, s2: s1.update(s2) or s1, + lambda s: s.top(20) + ), } Aggs = dict(SimpleAggs) @@ -62,12 +65,16 @@ Globals = {} Globals.update(Aggs) import math + Globals.update(math.__dict__) __eval = eval + + def eval(code, g={}, l={}): return __eval(code, g or Globals, l) + def table_join(f): def _join(self, other, left_keys=None, right_keys=None): if not left_keys: @@ -80,12 +87,16 @@ def _join(self, other, left_keys=None, right_keys=None): ln = [n for n in self.fields if n not in left_keys] rn = [n for n in other.fields if n not in right_keys] - def conv((k, (v1, v2))): - return list(k) + (v1 or [None]*len(ln)) + (v2 or [None]*len(rn)) + + def conv(t): + (k, (v1, v2)) = t + return list(k) + (v1 or [None] * len(ln)) + (v2 or [None] * len(rn)) + return joined.map(conv).asTable(left_keys + ln + rn, self.name) return _join + class TableRDD(DerivedRDD): def __init__(self, rdd, fields, name='', field_types=None): DerivedRDD.__init__(self, rdd) @@ -94,13 +105,11 @@ def __init__(self, rdd, fields, name='', field_types=None): fields = [n.strip() for n in fields.split(',')] self.fields = fields self.field_types = field_types - - def __str__(self): - return '' % (','.join(self.fields), self.prev) + self._repr = '' % (','.join(fields), rdd) def iterator(self, split): cls = namedtuple(self.name or ('Row%d' % self.id), self.fields) - return itertools.imap(lambda x:cls(*x), super(TableRDD, self).iterator(split)) + return map(lambda x: cls(*x), super(TableRDD, self).iterator(split)) def compute(self, split): return self.prev.iterator(split) @@ -143,12 +152,12 @@ def _create_field_name(self, e): def select(self, *fields, **named_fields): if len(fields) == 1 and not named_fields and fields[0] == '*': fields = self.fields - new_fields = [self._create_field_name(e) for e in fields] + named_fields.keys() + new_fields = [self._create_field_name(e) for e in fields] + list(named_fields.keys()) if len(set(new_fields)) != len(new_fields): raise Exception("dupicated fields: " + (','.join(new_fields))) selector = [self._create_expression(e) for e in fields] + \ - [self._create_expression(named_fields[n]) for n in new_fields[len(fields):]] + [self._create_expression(named_fields[n]) for n in new_fields[len(fields):]] _select = eval('lambda _v:(%s,)' % (','.join(e for e in selector))) need_attr = any(callable(f) for f in named_fields.values()) @@ -167,7 +176,7 @@ def _create_reducer(self, index, e): ag = '_x[%d]' % index if func_name in SimpleAggs: return (args, '%s(%s,%s)' % (func_name, ag, args), - '%s(_x[%d],_y[%d])' % (func_name, index, index), ag) + '%s(_x[%d],_y[%d])' % (func_name, index, index), ag) elif func_name in FullAggs: return ('%s[0](%s)' % (func_name, args), '%s[1](%s, %s)' % (func_name, ag, args), @@ -175,18 +184,18 @@ def _create_reducer(self, index, e): '%s[3](_x[%d])' % (func_name, index),) elif func_name: raise Exception("invalid aggregator function: %s" % func_name) - else: # group by + else: # group by return ('[%s]' % args, '_x[%d].append(%s) or _x[%d]' % (index, args, index), '_x[%d] + _y[%d]' % (index, index), ag) def selectOne(self, *fields, **named_fields): - new_fields = [self._create_field_name(e) for e in fields] + named_fields.keys() + new_fields = [self._create_field_name(e) for e in fields] + list(named_fields.keys()) if len(set(new_fields)) != len(new_fields): raise Exception("dupicated fields: " + (','.join(new_fields))) - codes = ([self._create_reducer(i, e) for i,e in enumerate(fields)] - + [self._create_reducer(i + len(fields), named_fields[n]) - for i,n in enumerate(new_fields[len(fields):])]) + codes = ([self._create_reducer(i, e) for i, e in enumerate(fields)] + + [self._create_reducer(i + len(fields), named_fields[n]) + for i, n in enumerate(new_fields[len(fields):])]) creater = eval('lambda _v:(%s,)' % (','.join(c[0] for c in codes))) merger = eval('lambda _x, _v:(%s,)' % (','.join(c[1] for c in codes))) @@ -202,6 +211,7 @@ def reducePartition(it): else: r = merger(r, i) return r + rs = self.ctx.runJob(self.prev, reducePartition) return [mapper(reduce(combiner, (x for x in rs if x is not None)))] @@ -223,17 +233,17 @@ def groupBy(self, keys, *fields, **kw): expr = ','.join(self._create_expression(e) for e in keys) gen_key = eval('lambda _v:(%s,)' % expr) - values = [self._create_field_name(e) for e in fields] + kw.keys() + values = [self._create_field_name(e) for e in fields] + list(kw.keys()) kw.update((values[i], fields[i]) for i in range(len(fields))) - codes = [self._create_reducer(i, kw[n]) for i,n in enumerate(values)] + codes = [self._create_reducer(i, kw[n]) for i, n in enumerate(values)] creater = eval('lambda _v:(%s,)' % (','.join(c[0] for c in codes))) merger = eval('lambda _x, _v:(%s,)' % (','.join(c[1] for c in codes))) combiner = eval('lambda _x, _y:(%s,)' % (','.join(c[2] for c in codes))) mapper = eval('lambda _x:(%s,)' % ','.join(c[3] for c in codes)) agg = Aggregator(creater, merger, combiner) - g = self.prev.map(lambda v:(gen_key(v), v)).combineByKey(agg, numSplits) - return g.map(lambda (k,v): k + mapper(v)).asTable(key_names + values, self.name) + g = self.prev.map(lambda v: (gen_key(v), v)).combineByKey(agg, numSplits) + return g.map(lambda k_v1: k_v1[0] + mapper(k_v1[1])).asTable(key_names + values, self.name) def indexBy(self, keys=None): if keys is None: @@ -244,9 +254,12 @@ def indexBy(self, keys=None): def pick(keys, fields): ki = [fields.index(n) for n in keys] vi = [i for i in range(len(fields)) if fields[i] not in keys] + def _(v): return tuple(v[i] for i in ki), [v[i] for i in vi] + return _ + return self.prev.map(pick(keys, self.fields)) @table_join @@ -264,8 +277,8 @@ def outerJoin(self, other, left_keys=None, right_keys=None): @table_join def leftOuterJoin(self, other, left_keys=None, right_keys=None): o = other.indexBy(right_keys).collectAsMap() - r = self.indexBy(left_keys).map(lambda (k,v):(k,(v,o.get(k)))) - r.mem += (sys.getsizeof(o) * 10) >> 20 # memory used by broadcast obj + r = self.indexBy(left_keys).map(lambda k_v: (k_v[0], (k_v[1], o.get(k_v[0])))) + r.mem += (sys.getsizeof(o) * 10) >> 20 # memory used by broadcast obj return r @table_join @@ -277,10 +290,11 @@ def sort(self, fields, reverse=False, numSplits=None): keys = [self.fields.index(fields)] else: keys = [self.fields.index(n) for n in fields] + def key(v): return tuple(v[i] for i in keys) - if len(self) <= 16: # maybe grouped + if len(self) <= 16: # maybe grouped data = sorted(self.prev.collect(), key=key, reverse=reverse) return self.ctx.makeRDD(data).asTable(self.fields, self.name) @@ -288,8 +302,10 @@ def key(v): def top(self, n, fields, reverse=False): keys = [self.fields.index(i) for i in fields] + def key(v): return tuple(v[i] for i in keys) + return self.prev.top(n, key=key, reverse=reverse) def collect(self): @@ -299,9 +315,10 @@ def take(self, n): return self.prev.take(n) def execute(self, sql, asTable=False): - sql_p = re.compile(r'(select|from|(?:inner|left outer)? join(?: each)?|where|group by|having|order by|limit) ', re.I) + sql_p = re.compile(r'(select|from|(?:inner|left outer)? join(?: each)?|where|group by|having|order by|limit) ', + re.I) parts = [i.strip() for i in sql_p.split(sql)[1:]] - kw = dict(zip([i.lower() for i in parts[::2]], parts[1::2])) + kw = dict(list(zip([i.lower() for i in parts[::2]], parts[1::2]))) for type in kw: if 'join' not in type: @@ -340,7 +357,7 @@ def execute(self, sql, asTable=False): r = self # select needed cols - #r = self.select(*[n for n in self.fields if n in sql]) + # r = self.select(*[n for n in self.fields if n in sql]) cols = [n.strip() for n in self._split_expr(kw['select'])] if 'where' in kw: @@ -387,14 +404,17 @@ def execute(self, sql, asTable=False): def save(self, path, overwrite=True, compress=True): r = OutputTableFileRDD(self.prev, path, - overwrite=overwrite, compress=compress).collect() - open(os.path.join(path, '.field_names'), 'w').write('\t'.join(self.fields)) + overwrite=overwrite, compress=compress).collect() + + with open(os.path.join(path, '.field_names'), 'w') as f: + f.write('\t'.join(self.fields)) return r CachedTables = { } + def create_table(ctx, name, expr): if name in CachedTables: return CachedTables[name] @@ -416,7 +436,7 @@ def create_table(ctx, name, expr): elif ',' in row: t = t.fromCsv(dialet='excel') else: - t = t.map(lambda line:line.strip().split(' ')) + t = t.map(lambda line: line.strip().split(' ')) # fake fields names row = t.first() @@ -424,4 +444,3 @@ def create_table(ctx, name, expr): t = t.asTable(fields, name) CachedTables[name] = t return t - diff --git a/dpark/tabular.py b/dpark/tabular.py index 241e9a41..b8667287 100644 --- a/dpark/tabular.py +++ b/dpark/tabular.py @@ -1,17 +1,20 @@ +from __future__ import absolute_import import os import zlib import types import socket import struct import marshal -import cPickle -from lz4 import compress, decompress from dpark.rdd import RDD, MultiSplit, TextFileRDD, Split, ParallelCollection, cached -from dpark.util import chain -from dpark.moosefs import walk -from dpark.bitindex import Bloomfilter, BitIndex +from dpark.utils import chain, atomic_file +from dpark.file_manager import walk +from dpark.utils.bitindex import Bloomfilter, BitIndex from dpark.serialize import dumps, loads from dpark.dependency import OneToOneDependency, OneToRangeDependency +from contextlib import closing +import six +from six.moves import map, range, zip, cPickle +from dpark.utils.lz4wrapper import compress, decompress ''' Strip Format: @@ -44,11 +47,13 @@ BITMAP_INDEX = 0 BLOOMFILTER_INDEX = 1 + class NamedTuple(object): _fields = [] _values = () + def __init__(self, fields, values): - if isinstance(fields, types.StringTypes): + if isinstance(fields, (str,)): fields = fields.replace(',', ' ').split() fields = list(fields) @@ -88,6 +93,7 @@ def __iter__(self): def __contains__(self, item): return item in self._values + class AdaptiveIndex(object): def __init__(self): self.index = {} @@ -122,36 +128,43 @@ def filter(self, fun): if self.index_type == BITMAP_INDEX: return chain(v.positions() for k, v in self.index.items() if fun(k)) + class TabularSplit(Split): def __init__(self, index, rdd, sp): self.index = index self.rdd = rdd self.split = sp + class TabularRDD(RDD): - def __init__(self, ctx, path, fields = None, taskMemory=None): + def __init__(self, ctx, path, fields=None, taskMemory=None): RDD.__init__(self, ctx) if taskMemory: self.mem = taskMemory - if isinstance(path, basestring): + if isinstance(path, six.string_types): files = self._get_files(path) else: files = chain(self._get_files(p) for p in path) - self.rdds = [TabularFileRDD(ctx, f, fields) for f in files] + rdds = [TabularFileRDD(ctx, f, fields) for f in files] self._splits = [] i = 0 - for rdd in self.rdds: + for rdd in rdds: for sp in rdd.splits: self._splits.append(TabularSplit(i, rdd, sp)) i += 1 - self.dependencies = [OneToOneDependency(rdd) for rdd in self.rdds] + self._dependencies = [OneToOneDependency(rdd) for rdd in rdds] + self.repr_name = '<%s %d %s...>' % (self.__class__.__name__, len(rdds), + ','.join(str(rdd) for rdd in rdds[:1])) + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = split.rdd.preferredLocations(split.split) def _get_files(self, path): path = os.path.realpath(path) if os.path.isdir(path): - for root,dirs,names in walk(path): + for root, dirs, names in walk(path): for n in sorted(names): if not n.startswith('.'): yield os.path.join(root, n) @@ -159,33 +172,29 @@ def _get_files(self, path): else: yield path - def __repr__(self): - return '<%s %d %s...>' % (self.__class__.__name__, len(self.rdds), - ','.join(str(rdd) for rdd in self.rdds[:1])) - - def _preferredLocations(self, split): - return split.rdd.preferredLocations(split.split) - def compute(self, split): return split.rdd.iterator(split.split) def filterByIndex(self, **kw): return FilteredByIndexRDD(self, kw) + class FilteredByIndexRDD(RDD): def __init__(self, rdd, filters): RDD.__init__(self, rdd.ctx) self.rdd = rdd self.filters = filters self.mem = max(self.mem, rdd.mem) - self.dependencies = [OneToOneDependency(rdd)] + self._dependencies = [OneToOneDependency(rdd)] self._splits = self._get_splits() + self.repr_name = '<%s %s>' % (self.__class__.__name__, rdd) + self._preferred_locs = {} + for split in self._splits: + self._preferred_locs[split] = rdd.preferredLocations(split) - def __repr__(self): - return '<%s %s>' % (self.__class__.__name__, self.rdd) - - def _preferredLocations(self, split): - return self.rdd.preferredLocations(split) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdd = None @cached def __getstate__(self): @@ -199,16 +208,17 @@ def __setstate__(self, state): def _get_splits(self): filters = self.filters + def _filter(v): - path, splits = v - result_set = set(sp.split.begin / STRIPE_SIZE for sp in splits) + path, ids = v + result_set = set(ids) with open(path, 'rb') as f: f.seek(-8, 2) footer_fields_size, footer_indices_size = struct.unpack('II', f.read(8)) - f.seek(-8 -footer_fields_size -footer_indices_size, 2) + f.seek(-8 - footer_fields_size - footer_indices_size, 2) indices = cPickle.loads(zlib.decompress(f.read(footer_indices_size))) _fields = marshal.loads(decompress(f.read(footer_fields_size))) - for k, v in filters.iteritems(): + for k, v in six.iteritems(filters): result = set() if k not in _fields: raise RuntimeError('%s is not in fields!' % k) @@ -228,31 +238,34 @@ def _filter(v): v = [v] for vv in v: - for id in result_set: - if index.get(vv, id): - result.add(id) - - result_set &= result - - for sp in splits: - id = sp.split.begin / STRIPE_SIZE - if id in result: - yield sp + for _id in result_set: + if index.get(vv, _id): + result.add(_id) + return path, result sp_dict = {} for sp in self.rdd.splits: path = sp.rdd.path + _id = sp.split.begin / STRIPE_SIZE if path not in sp_dict: - sp_dict[path] = [sp] + sp_dict[path] = [_id] else: - sp_dict[path].append(sp) + sp_dict[path].append(_id) - rdd = ParallelCollection(self.ctx, sp_dict.items(), len(sp_dict)) - return rdd.flatMap(_filter).collect() + rdd = ParallelCollection(self.ctx, list(sp_dict.items()), len(sp_dict)) + path_ids_filter = dict(rdd.map(_filter).collect()) + + splits = [] + for sp in self.rdd.splits: + path = sp.rdd.path + _id = sp.split.begin / STRIPE_SIZE + if _id in path_ids_filter[path]: + splits.append(sp) + return splits def compute(self, split): for t in self.rdd.iterator(split): - for k, v in self.filters.iteritems(): + for k, v in six.iteritems(self.filters): value = getattr(t, k) if isinstance(v, types.FunctionType): if not v(value): @@ -267,17 +280,18 @@ def compute(self, split): else: yield t + class TabularFileRDD(TextFileRDD): - def __init__(self, ctx, path, fields = None): - TextFileRDD.__init__(self, ctx, path, splitSize = STRIPE_SIZE) - if isinstance(fields, basestring): + def __init__(self, ctx, path, fields=None): + TextFileRDD.__init__(self, ctx, path, splitSize=STRIPE_SIZE) + if isinstance(fields, six.string_types): fields = fields.replace(',', ' ').split() - self.fields = map(str, fields) if fields is not None else None + self.fields = list(map(str, fields)) if fields is not None else None def compute(self, split): - with self.open_file() as f: - f.seek(-8,2) + with closing(self.open_file()) as f: + f.seek(-8, 2) footer_fields_size, footer_indices_size = struct.unpack('II', f.read(8)) footer_offset = self.size - 8 - footer_fields_size - footer_indices_size footer_fields_offset = self.size - 8 - footer_fields_size @@ -291,7 +305,7 @@ def compute(self, split): _fields = marshal.loads(decompress(f.read(footer_fields_size))) if self.fields is None: - field_ids = range(len(_fields)) + field_ids = list(range(len(_fields))) field_names = _fields else: field_names = [] @@ -318,6 +332,7 @@ def compute(self, split): for r in zip(*content): yield NamedTuple(field_names, r) + class OutputTabularRDD(RDD): def __init__(self, rdd, path, field_names, indices=None, numSplits=None): RDD.__init__(self, rdd.ctx) @@ -328,14 +343,14 @@ def __init__(self, rdd, path, field_names, indices=None, numSplits=None): raise RuntimeError('path already exists: %s' % path) os.makedirs(path) - if isinstance(field_names, basestring): + if isinstance(field_names, six.string_types): field_names = field_names.replace(',', ' ').split() if len(set(field_names)) != len(field_names): raise ValueError('duplicated field names') - self.fields = map(str, field_names) - if isinstance(indices, types.StringTypes): + self.fields = list(map(str, field_names)) + if isinstance(indices, (str,)): indices = indices.replace(',', ' ').split() self.indices = set() @@ -350,20 +365,20 @@ def __init__(self, rdd, path, field_names, indices=None, numSplits=None): prev_splits = len(rdd) numSplits = min(numSplits or prev_splits, prev_splits) self.numSplits = min(numSplits, prev_splits) - s = [int(round(1.0*prev_splits/numSplits*i)) for i in xrange(numSplits + 1)] - self._splits = [MultiSplit(i, rdd.splits[s[i]:s[i+1]]) for i in xrange(numSplits)] - self.dependencies = [OneToRangeDependency(rdd, int(prev_splits/numSplits), - prev_splits)] + s = [int(round(1.0 * prev_splits / numSplits * i)) for i in range(numSplits + 1)] + self._splits = [MultiSplit(i, rdd.splits[s[i]:s[i + 1]]) for i in range(numSplits)] + self._dependencies = [OneToRangeDependency(rdd, int(prev_splits / numSplits), + prev_splits)] + self.repr_name = '' % (path, rdd) - def __repr__(self): - return '' % (self.path, self.prev) + def _clear_dependencies(self): + RDD._clear_dependencies(self) + self.rdd = None def compute(self, split): buffers = [list() for i in self.fields] remain_size = STRIPE_DATA_SIZE path = os.path.join(self.path, '%04d.dt' % split.index) - tpath = os.path.join(self.path, '.%04d_%s_%s.dt' % ( - split.index, socket.gethostname(), os.getpid())) indices = dict((i, AdaptiveIndex()) for i in self.indices) def write_stripe(f, compressed, header, padding=True): @@ -379,54 +394,42 @@ def write_stripe(f, compressed, header, padding=True): if padding: f.write('\0' * padding_size) - try: - with open(tpath, 'wb+') as f: - stripe_id = 0 - for it in chain(self.prev.iterator(sp) for sp in split.splits): - row = it[:len(self.fields)] - size = len(marshal.dumps(tuple(row))) - if size > STRIPE_DATA_SIZE: - raise RuntimeError('Row too big') - - if size > remain_size: - compressed = [compress(marshal.dumps(tuple(b))) for b in buffers] - _sizes = tuple(map(len, compressed)) - _remain_size = STRIPE_DATA_SIZE - sum(_sizes) - if size > _remain_size: - write_stripe(f, compressed, _sizes) - buffers = [list() for i in self.fields] - remain_size = STRIPE_DATA_SIZE - stripe_id += 1 - else: - remain_size = _remain_size - - remain_size -= size - for i, value in enumerate(row): - buffers[i].append(value) - field = self.fields[i] - if field in self.indices: - indices[field].add(value, stripe_id) + with atomic_file(path) as f: + stripe_id = 0 + for it in chain(self.prev.iterator(sp) for sp in split.splits): + row = it[:len(self.fields)] + size = len(marshal.dumps(tuple(row))) + if size > STRIPE_DATA_SIZE: + raise RuntimeError('Row too big') - if any(buffers): + if size > remain_size: compressed = [compress(marshal.dumps(tuple(b))) for b in buffers] _sizes = tuple(map(len, compressed)) - write_stripe(f, compressed, _sizes, False) - - footer_indices = zlib.compress(cPickle.dumps(indices, -1)) - footer_fields = compress(marshal.dumps(self.fields)) - f.write(footer_indices) - f.write(footer_fields) - f.write(struct.pack('II', len(footer_fields), len(footer_indices))) - - try: - os.rename(tpath, path) - except OSError: - pass - - yield path - finally: - try: - os.remove(tpath) - except OSError: - pass - + _remain_size = STRIPE_DATA_SIZE - sum(_sizes) + if size > _remain_size: + write_stripe(f, compressed, _sizes) + buffers = [list() for i in self.fields] + remain_size = STRIPE_DATA_SIZE + stripe_id += 1 + else: + remain_size = _remain_size + + remain_size -= size + for i, value in enumerate(row): + buffers[i].append(value) + field = self.fields[i] + if field in self.indices: + indices[field].add(value, stripe_id) + + if any(buffers): + compressed = [compress(marshal.dumps(tuple(b))) for b in buffers] + _sizes = tuple(map(len, compressed)) + write_stripe(f, compressed, _sizes, False) + + footer_indices = zlib.compress(cPickle.dumps(indices, -1)) + footer_fields = compress(marshal.dumps(self.fields)) + f.write(footer_indices) + f.write(footer_fields) + f.write(struct.pack('II', len(footer_fields), len(footer_indices))) + + yield path diff --git a/dpark/task.py b/dpark/task.py index 63af9265..e7d89f8e 100644 --- a/dpark/task.py +++ b/dpark/task.py @@ -1,130 +1,457 @@ -import os,os.path -import socket +from __future__ import absolute_import import marshal -import cPickle -import logging -import struct +import time +import six +from six.moves import range, cPickle +import os +import os.path -from dpark.util import compress, decompress -from dpark.serialize import marshalable, load_func, dump_func -from dpark.shuffle import LocalFileShuffle +import dpark.conf +from dpark.env import env +from dpark.utils import compress, DparkUserFatalError +from dpark.utils.memory import ERROR_TASK_OOM +from dpark.utils.log import get_logger +from dpark.serialize import marshalable, load_func, dump_func, dumps, loads +from dpark.shuffle import get_serializer, Merger, pack_header, ShuffleWorkDir -logger = logging.getLogger(__name__) +logger = get_logger(__name__) -class Task: - def __init__(self): - self.id = Task.newId() - nextId = 0 - @classmethod - def newId(cls): - cls.nextId += 1 - return cls.nextId +class TTID(object): + """"Task Try ID - def run(self, id): - raise NotImplementedError + 1.2_3.4: ttid - def preferredLocations(self): - raise NotImplementedError + 1: stage id, start from 1 + 2: stage try counter (for fetch fail), start from 1 + 3: task partition of the stage, start from 0 + 4: task retry counter + + 1.2: taskset id + 1.2_3: task id + """ + + def __init__(self, ttid): + self.ttid = ttid + self.taskset_id, part_try = ttid.split("_") + self.stage_id, self.stage_try = list(map(int, self.taskset_id.split("."))) + self.part, self.task_try = list(map(int, part_try.split("."))) + self.task_id = ttid.rsplit(".", 1)[0] + + @staticmethod + def make_taskset_id(stage_id, stage_num_try): + return "{}.{}".format(stage_id, stage_num_try) + + @staticmethod + def make_task_id(taskset_id, partition): + return "{}_{}".format(taskset_id, partition) + + @staticmethod + def make_ttid(task_id, task_num_try): + return "{}.{}".format(task_id, task_num_try) + + +class TaskTry(object): + + def __init__(self, reason): + self.reason = reason + self.status = [(TaskState.staging, time.time())] + def append(self, st): + self.status.append((st, time.time())) -class DAGTask(Task): - def __init__(self, stageId): - Task.__init__(self) - self.stageId = stageId + def __str__(self): + return self.reason + ":" + ",".join(list(map(lambda x: "%s@%s" % (x[0], int(x[1])), self.status))) + + +class DAGTask(object): + def __init__(self, stage_id, taskset_id, partition): + self.id = TTID.make_task_id(taskset_id, partition) + self.stage_id = stage_id + self.taskset_id = taskset_id + self.partition = partition + self.num_try = 0 + self.reason_next = TaskReason.first + self.tries = {} + + self.status = None + self.time_used = 0 # sum up time of mulity retry + + self.mem = 0 + self.cpus = 0 + self.gpus = 0 + + self.stage_time = 0 + self.start_time = 0 def __repr__(self): - return ''%(self.stageId, self.id) + return ''.format(self.id) + + @property + def try_id(self): + return TTID.make_ttid(self.id, self.num_try) + + def try_next(self): + self.num_try += 1 + self.tries[self.num_try] = TaskTry(self.reason_next) + + def update_status(self, status, num_try): + self.status = status + self.tries[num_try].append(status) + + def run(self, task_try_id): + try: + if self.mem != 0: + env.meminfo.start(task_try_id, int(self.mem)) + if dpark.conf.MULTI_SEGMENT_DUMP: + env.meminfo.check = False + return self._run(task_try_id) + except KeyboardInterrupt as e: + if self.mem != 0 and env.meminfo.oom: + os._exit(ERROR_TASK_OOM) + else: + raise e + finally: + if self.mem != 0: + env.meminfo.check = True + env.meminfo.stop() + + def _run(self, task_try_id): + raise NotImplementedError + + def preferredLocations(self): + raise NotImplementedError class ResultTask(DAGTask): - def __init__(self, stageId, rdd, func, partition, locs, outputId): - DAGTask.__init__(self, stageId) + def __init__(self, stage_id, taskset_id, partition, rdd, func, locs, outputId): + DAGTask.__init__(self, stage_id, taskset_id, partition) self.rdd = rdd self.func = func - self.partition = partition self.split = rdd.splits[partition] self.locs = locs self.outputId = outputId - def run(self, attemptId): - logger.debug("run task %s with %d", self, attemptId) - return self.func(self.rdd.iterator(self.split)) + def _run(self, task_id): + logger.debug("run task %s: %s", task_id, self) + t0 = time.time() + res = self.func(self.rdd.iterator(self.split)) + env.task_stats.secs_all = time.time() - t0 + return res def preferredLocations(self): return self.locs def __repr__(self): - return "' % (self.shuffleId, self.partition, self.rdd) + shuffleId = getattr(self, 'shuffleId', None) + partition = getattr(self, 'partition', None) + rdd = getattr(self, 'rdd', None) + return '' % (shuffleId, partition, rdd) + + def __getstate__(self): + d = dict(self.__dict__) + del d['rdd'] + del d['split'] + return d, dumps(self.rdd), dumps(self.split) + + def __setstate__(self, state): + d, rdd, split = state + self.__dict__.update(d) + self.rdd = loads(rdd) + self.split = loads(split) def preferredLocations(self): return self.locs - def run(self, attempId): - logger.debug("shuffling %d of %s", self.partition, self.rdd) - numOutputSplits = self.partitioner.numPartitions - getPartition = self.partitioner.getPartition - mergeValue = self.aggregator.mergeValue - createCombiner = self.aggregator.createCombiner - - buckets = [{} for i in range(numOutputSplits)] - for k,v in self.rdd.iterator(self.split): - bucketId = getPartition(k) - bucket = buckets[bucketId] - r = bucket.get(k, None) - if r is not None: - bucket[k] = mergeValue(r, v) - else: - bucket[k] = createCombiner(v) + def _run(self, task_id): + mem_limit = env.meminfo.mem_limit_soft + t0 = time.time() + logger.debug("run task with shuffle_flag %r" % (self.rddconf,)) + rdd = self.rdd + meminfo = env.meminfo + n = self.partitioner.numPartitions + get_partition = self.partitioner.getPartition + merge_value = self.aggregator.mergeValue + create_combiner = self.aggregator.createCombiner + dumper_cls = SortMergeBucketDumper if self.rddconf.sort_merge else BucketDumper + dumper = dumper_cls(self.shuffleId, self.partition, n, self.rddconf) + buckets = [{} for _ in range(n)] + env.meminfo.ratio = min(float(n) / (n + 1), env.meminfo.ratio) - for i in range(numOutputSplits): + last_i = 0 + for i, item in enumerate(rdd.iterator(self.split)): try: - if marshalable(buckets[i]): - flag, d = 'm', marshal.dumps(buckets[i]) - else: - flag, d = 'p', cPickle.dumps(buckets[i], -1) - except ValueError: - flag, d = 'p', cPickle.dumps(buckets[i], -1) - cd = compress(d) - for tried in range(1, 4): try: - path = LocalFileShuffle.getOutputFile(self.shuffleId, self.partition, i, len(cd) * tried) - tpath = path + ".%s.%s" % (socket.gethostname(), os.getpid()) - f = open(tpath, 'wb', 1024*4096) - f.write(flag + struct.pack("I", 5 + len(cd))) - f.write(cd) - f.close() - os.rename(tpath, path) - break - except IOError, e: - logger.warning("write %s failed: %s, try again (%d)", path, e, tried) - try: os.remove(tpath) - except OSError: pass - else: + k, v = item + except (TypeError, ValueError) as e: + msg = "item of {} should be (k, v) pair, got: {}, exception: {}".format(rdd.scope.key, item, e) + raise DparkUserFatalError(msg) + + bucket = buckets[get_partition(k)] + r = bucket.get(k, None) + if r is not None: + bucket[k] = merge_value(r, v) + else: + bucket[k] = create_combiner(v) + + if dpark.conf.MULTI_SEGMENT_DUMP and meminfo.rss > mem_limit: + _log = logger.info if dpark.conf.LOG_ROTATE else logger.debug + _log("dump rotate %d with %d kv: mem %d MB, sort limit %d MB, limit %d MB", + env.task_stats.num_dump_rotate + 1, + i - last_i, + int(meminfo.rss) >> 20, + mem_limit >> 20, + int(meminfo.mem) >> 20) + dumper.dump(buckets, False) + [buckets[j].clear() for j in range(n)] + env.meminfo.after_rotate() + mem_limit = env.meminfo.mem_limit_soft + last_i = i + except ValueError as e: + logger.exception('The ValueError exception: %s at %s', str(e), str(rdd.scope.api_callsite)) raise - return LocalFileShuffle.getServerUri() + t1 = time.time() + dumper.dump(buckets, True) + dumper.commit(self.aggregator) + del buckets + env.task_stats.bytes_dump += dumper.get_size() + env.task_stats.num_dump_rotate += 1 + t = time.time() + env.task_stats.secs_dump += t - t1 + env.task_stats.secs_all = t - t0 + + return env.server_uri + + +class BucketDumperBase(object): + + def __init__(self, shuffle_id, map_id, num_reduce, rddconf): + self.shuffle_id = shuffle_id + self.map_id = map_id + self.num_reduce = n = num_reduce + self.rddconf = rddconf + self.paths = [ShuffleWorkDir(self.shuffle_id, self.map_id, i) for i in range(num_reduce)] + + self.tmp_paths = [[] for _ in range(n)] # last one is used for export + # stats + self.sizes = [0 for _ in range(n)] + self.num_dump = 0 + + def get_size(self): + return sum(self.sizes) + + def dump(self, buckets, is_final): + t = time.time() + for i, bucket_dict in enumerate(buckets): + if not bucket_dict: + continue + items = six.iteritems(bucket_dict) + data, exp_size = self._prepare(items) + tmppath = self._get_tmp(i, is_final, exp_size) + logger.debug("dump %s", tmppath) + size = self._dump_bucket(data, tmppath) + self.sizes[i] += size + + self.num_dump += 1 + t = time.time() - t + env.task_stats.secs_dump += t + env.task_stats.num_dump_rotate += 1 + + def commit(self, aggregator): + self._pre_commit(aggregator) + for i in range(self.num_reduce): + tmppaths = self.tmp_paths[i] + if tmppaths: + self.paths[i].export(tmppaths[-1]) + else: + self._dump_empty_bucket(i) + + def _dump_empty_bucket(self, i): + tmppath = self.paths[i].alloc_tmp() + self._dump_bucket(self._prepare([])[0], tmppath) + self.paths[i].export(tmppath) + + def _get_tmp(self, reduce_id, is_final, size): + pass + + +class BucketDumper(BucketDumperBase): + + def _get_tmp(self, reduce_id, is_final, size): + # each reduce has one tmp + # each tmp may be opened and appended multi times + + i = reduce_id + tmp_paths = self.tmp_paths[i] + if tmp_paths: + tmp_path = tmp_paths[0] + else: + if is_final and self.num_dump == 0: + tmp_path = ShuffleWorkDir.alloc_tmp(datasize=size) + else: + tmp_path = ShuffleWorkDir.alloc_tmp(mem_first=False) + tmp_paths.append(tmp_path) + + return tmp_path + + def _pre_commit(self, aggregator): + pass + + def _prepare(self, items): + items = list(items) + try: + if marshalable(items): + is_marshal, d = True, marshal.dumps(items) + else: + is_marshal, d = False, cPickle.dumps(items, -1) + except ValueError: + is_marshal, d = False, cPickle.dumps(items, -1) + data = compress(d) + size = len(data) + return (is_marshal, data), size + + def _dump_bucket(self, data, path): + is_marshal, data = data + if self.num_dump == 0 and os.path.exists(path): + logger.warning("remove old dump %s", path) + os.remove(path) + with open(path, 'ab') as f: + f.write(pack_header(len(data), is_marshal, False)) + f.write(data) + return len(data) + + +class SortMergeBucketDumper(BucketDumperBase): + + def _pre_commit(self, aggregator): + for i in range(self.num_reduce): + tmp_paths = self.tmp_paths[i] + if tmp_paths: + if len(tmp_paths) == 1: + self.paths[i].export(tmp_paths[0]) + else: + inputs = [get_serializer(self.rddconf).load_stream(open(p)) + for p in tmp_paths] + rddconf = self.rddconf.dup(op=dpark.conf.OP_GROUPBY) + merger = Merger.get(rddconf, aggregator=aggregator, api_callsite=self.__class__.__name__) + merger.merge(inputs) + final_tmp = self._get_tmp(i, True, 0) + with open(final_tmp, 'wb') as f: + get_serializer(self.rddconf).dump_stream(merger, f) + else: + self._dump_empty_bucket(i) + + def _get_tmp(self, i, is_final, size): + # each dump write to a new tmp file for each reduce + p = ShuffleWorkDir.alloc_tmp(mem_first=False) + self.tmp_paths[i].append(p) + return p + + def _prepare(self, items): + return items, None + + def _dump_bucket(self, items, path): + serializer = get_serializer(self.rddconf) + with open(path, 'wb') as f: + serializer.dump_stream(sorted(items), f) + size = f.tell() + return size + + +class TaskState: + # non terminal states + staging = 'TASK_STAGING' + running = 'TASK_RUNNING' + + # terminal states + finished = 'TASK_FINISHED' + failed = 'TASK_FAILED' + killed = 'TASK_KILLED' + lost = 'TASK_LOST' + error = 'TASK_ERROR' + + +class TaskEndReason: + # generated on the executor + success = 'FINISHED_SUCCESS' + other_ecs = 'FAILED_UNKNOWN_EXITCODE' + load_failed = 'FAILED_PICKLE_LOAD' + other_failure = 'FAILED_OTHER_FAILURE' + fetch_failed = 'FAILED_FETCH_FAILED' + task_oom = 'FAILED_TASK_OOM' + recv_sig = 'FAILED_RECV_SIG' + recv_sig_kill = 'FAILED_RECV_SIG_KILL' + launch_failed = 'FAILED_LAUNCH_FAILED' + + # generated on the agent + mesos_cgroup_oom = 'REASON_CONTAINER_LIMITATION_MEMORY' + + @classmethod + def maybe_oom(cls, reason): + return reason in (cls.task_oom, cls.recv_sig_kill, cls.mesos_cgroup_oom) + + +class TaskReason: + first = "first" + run_timeout = "run_timeout" + stage_timeout = "stage_timout" + fail = "fail" + + +class FetchFailed(Exception): + + def __init__(self, serverUri, shuffleId, mapId, reduceId): + self.serverUri = serverUri + self.shuffleId = shuffleId + self.mapId = mapId + self.reduceId = reduceId + + def __str__(self): + return '' % ( + self.serverUri, self.shuffleId, self.mapId, self.reduceId + ) + + def __reduce__(self): + return FetchFailed, (self.serverUri, self.shuffleId, + self.mapId, self.reduceId) + + +class OtherFailure(Exception): + + def __init__(self, message): + self.message = message + + def __str__(self): + return '' % self.message \ No newline at end of file diff --git a/dpark/taskset.py b/dpark/taskset.py new file mode 100644 index 00000000..9bc1b244 --- /dev/null +++ b/dpark/taskset.py @@ -0,0 +1,452 @@ +from __future__ import absolute_import, print_function +import math +import time +import socket +from operator import itemgetter + +from dpark.utils.tdigest import TDigest +from dpark.utils.log import ( + get_logger, make_progress_bar +) +from dpark.hostatus import TaskHostManager +from dpark.task import TaskState, TaskEndReason, TaskReason +from six.moves import range + +logger = get_logger(__name__) + + +def readable(size): + units = ['B', 'KB', 'MB', 'GB', 'TB', 'PB'] + unit = 0 + while size > 1024: + size /= 1024.0 + unit += 1 + return '%.1f%s' % (size, units[unit]) + + +LOCALITY_WAIT = 0 +WAIT_FOR_RUNNING = 30 +MAX_TASK_FAILURES = 4 +MAX_TASK_MEMORY = 20 << 10 # 20GB + + +class TaskCounter(object): + + def __init__(self, n): + self.n = n + self.launched = 0 + self.finished = 0 + + self.fail_oom = 0 + self.fail_fetch = 0 + self.fail_run_timeout = 0 + self.fail_staging_timeout = 0 + self.fail_all = 0 # include oom, not include timeout + + @property + def running(self): + return self.launched - self.finished + + def get_fail_types(self): + prefix = "fail" + return [a for a in self.__dict__ if a.startswith(prefix)] + + +class TaskSet(object): + """ A TaskSet runs a set of tasks of a Stage with retry. + + - Task_id seen by TaskSet not include task.num_try + - Each task try four times before abort. + - Enlarge task.mem if fail for OOM. + - Retry for lagging tasks. + """ + + def __init__(self, sched, tasks, cpus=1, mem=100, gpus=0, + task_host_manager=None): + self.start_time = time.time() + self.sched = sched + self.tasks = tasks + self.id = tasks[0].taskset_id + self.ttids = set() + + for t in self.tasks: + t.status = None + t.num_try = 0 + t.time_used = 0 + t.cpus = cpus + t.mem = mem + t.gpus = gpus + + self.launched = [False] * len(tasks) + self.finished = [False] * len(tasks) + self.numFailures = [0] * len(tasks) + self.running_hosts = [[] for _ in range(len(tasks))] + self.tidToIndex = {} + self.counter = TaskCounter(len(tasks)) + + self.total_time_used = 0 + self.max_task_time = 0 + + self.lastPreferredLaunchTime = time.time() + + self.pendingTasksForHost = {} + self.pendingTasksWithNoPrefs = [] + self.allPendingTasks = [] + + self.reasons = set() + self.failed = False + self.causeOfFailure = '' + self.last_check = 0 + + for i in range(len(tasks)): + self._addPendingTask(i) + self.host_cache = {} + self.task_host_manager = task_host_manager if task_host_manager is not None \ + else TaskHostManager() + self.id_retry_host = {} + self.task_local_set = set() + self.mem_digest = TDigest() + self.max_stage_time = 0 + self.mem90 = 0 # TODO: move to stage + + @property + def taskEverageTime(self): + if not self.counter.finished: + return 10 + return max(self.total_time_used / self.counter.finished, 5) + + def _addPendingTask(self, i): + loc = self.tasks[i].preferredLocations() + if not loc: + self.pendingTasksWithNoPrefs.append(i) + else: + for host in loc: + self.pendingTasksForHost.setdefault(host, []).append(i) + self.allPendingTasks.append(i) + + def _getPendingTasksForHostWithCache(self, host): + tasks = self.host_cache.get(host) + if tasks: + return tasks + else: + tasks = self._getPendingTasksForHost(host) + self.host_cache[host] = tasks + return tasks + + def _getPendingTasksForHost(self, host): + try: + h, hs, ips = socket.gethostbyname_ex(host) + except Exception: + h, hs, ips = host, [], [] + tasks = sum((self.pendingTasksForHost.get(h, []) + for h in [h] + hs + ips), []) + st = {} + for t in tasks: + st[t] = st.get(t, 0) + 1 + ts = sorted(list(st.items()), key=itemgetter(1), reverse=True) + return [t for t, _ in ts] + + def _findTaskFromList(self, l, host, cpus, mem, gpus): + for i in l: + if self.launched[i] or self.finished[i]: + continue + if host in self.running_hosts[i]: + continue + t = self.tasks[i] + if self.task_host_manager.task_failed_on_host(t.id, host): + continue + if t.cpus <= cpus + 1e-4 and t.mem <= mem and t.gpus <= gpus: + return i + + def taskOffer(self, host_offers, cpus, mems, gpus): + prefer_list = [] + for host in host_offers: + i, o = host_offers[host] + local_task = self._findTaskFromList( + self._getPendingTasksForHostWithCache(host), host, + cpus[i], mems[i], gpus[i]) + if local_task is not None: + result_tuple = self._try_update_task_offer(local_task, i, o, cpus, mems, gpus) + if result_tuple is None: + continue + prefer_list.append(result_tuple) + if prefer_list: + return prefer_list + for idx in range(len(self.tasks)): + if not self.launched[idx] and not self.finished[idx]: + i, o = self.task_host_manager.offer_choice(self.tasks[idx].id, host_offers, + self.running_hosts[idx]) + if i is None: + continue + result_tuple = self._try_update_task_offer(idx, i, o, cpus, mems, gpus) + if result_tuple: + return [result_tuple] + return [] + + def _try_update_task_offer(self, task_idx, i, o, cpus, mem, gpus): + t = self.tasks[task_idx] + if t.cpus <= cpus[i] + 1e-4 and t.mem <= mem[i] and t.gpus <= gpus[i]: + t.status = TaskState.staging + t.host = o.hostname + t.try_next() + self.id_retry_host[(t.id, t.num_try)] = o.hostname + logger.debug('Starting task %s on slave %s', + t.try_id, o.hostname) + self.tidToIndex[t.id] = task_idx + self.launched[task_idx] = True + self.counter.launched += 1 + self.running_hosts[task_idx].append(o.hostname) + host_set = set(self.tasks[task_idx].preferredLocations()) + if o.hostname in host_set: + self.task_local_set.add(t.id) + return i, o, t + return None + + def statusUpdate(self, task_id, num_try, status, reason=None, message=None, + result=None, update=None, stats=None): + logger.debug('taskset status update %s, status %s, reason %s', task_id, status, reason) + if task_id not in self.tidToIndex: + logger.error('invalid task_id: %s, status %s, reason %s', task_id, status, reason) + return + i = self.tidToIndex[task_id] + task = self.tasks[i] + task.update_status(status, num_try) + + if self.finished[i]: + if status == TaskState.finished: + logger.debug('Task %s is already finished, ignore it', task_id) + return + + # when checking, task been masked as not launched + if not self.launched[i]: + self.launched[i] = True + self.counter.launched += 1 + + if status == TaskState.running: + task.start_time = time.time() + self.max_stage_time = max(self.max_stage_time, task.start_time - task.stage_time) + elif status == TaskState.finished: + if stats: + self.mem_digest.add(stats.bytes_max_rss / (1024. ** 2)) + if task.tries[num_try].reason in (TaskReason.run_timeout, TaskReason.stage_timeout): + logger.warning("task timeout works: try %s finshed. History: %s", + num_try, ". ".join(map(str, task.tries.values()))) + + self._task_finished(task_id, num_try, result, update, stats) + else: # failed, killed, lost, error + self._task_lost(task_id, num_try, status, reason, message, exception=result) + + def progress(self, ending=''): + n = self.counter.n + ratio = self.counter.finished * 1. / n + bar = make_progress_bar(ratio) + if self.counter.finished: + elasped = time.time() - self.start_time + avg = self.total_time_used / self.counter.finished + eta = (n - self.counter.finished) * elasped / self.counter.finished + m, s = divmod(int(eta), 60) + h, m = divmod(m, 60) + + tmpl = 'taskset:%4s {{GREEN}}%s{{RESET}}%5.1f%% (% {width}s/% {width}s) ETA:% 2d:%02d:%02d AVG:%.1fs\x1b[K%s' + fmt = tmpl.format(width=int(math.log10(self.counter.n)) + 1) + + msg = fmt % ( + self.id, bar, ratio * 100, self.counter.finished, n, h, m, s, + avg, ending + ) + msg = msg.ljust(80) + logger.info(msg) + else: + + tmpl = 'taskset:%4s {{GREEN}}%s{{RESET}}%5.1f%% (% {width}s/% {width}s) ETA:--:--:-- AVG:N/A\x1b[K%s' + fmt = tmpl.format(width=int(math.log10(self.counter.n)) + 1) + + msg = fmt % (self.id, bar, ratio * 100, self.counter.finished, n, ending) + msg = msg.ljust(80) + logger.info(msg) + + def _task_finished(self, task_id, num_try, result, update, stats): + i = self.tidToIndex[task_id] + self.finished[i] = True + self.counter.finished += 1 + task = self.tasks[i] + hostname = self.id_retry_host[(task.id, num_try)] \ + if (task.id, num_try) in self.id_retry_host else task.host + task.time_used += time.time() - task.start_time + self.total_time_used += task.time_used + self.max_task_time = max(self.max_task_time, task.time_used) + if getattr(self.sched, 'color', False): + title = 'taskset %s: task %s finished in %.1fs (%d/%d) ' % ( + self.id, task_id, task.time_used, self.counter.finished, self.counter.n) + msg = '\x1b]2;%s\x07\x1b[1A' % title + logger.info(msg) + + self.sched.taskEnded(task, TaskEndReason.success, result, update, stats) + self.running_hosts[i] = [] + self.task_host_manager.task_succeed(task.id, hostname, + TaskEndReason.success) + + for t in range(task.num_try): + if t + 1 != num_try: + self.sched.killTask(task.id, t + 1) + + if self.counter.finished == self.counter.n: + ts = [t.time_used for t in self.tasks] + num_try = [t.num_try for t in self.tasks] + elasped = time.time() - self.start_time + logger.info('taskset %s finished in %.1fs: min=%.1fs, ' + 'avg=%.1fs, max=%.1fs, maxtry=%d, speedup=%.1f, local=%.1f%%', + self.id, elasped, min(ts), sum(ts) / len(ts), max(ts), + max(num_try), self.total_time_used / elasped, + len(self.task_local_set) * 100. / len(self.tasks) + ) + self.sched.tasksetFinished(self) + + def _task_lost(self, task_id, num_try, status, reason, message, exception=None): + index = self.tidToIndex[task_id] + task = self.tasks[index] + + if reason == TaskEndReason.fetch_failed and self.numFailures[index] >= 1: + self.counter.fail_fetch += 1 + logger.warning('Cancel task %s after fetch fail twice from %s', + task_id, exception.serverUri) + self.sched.taskEnded(self.tasks[index], reason, exception, None) + # cancel tasks + if not self.finished[index]: + self.finished[index] = True + self.counter.finished += 1 + for i in range(len(self.finished)): + if not self.launched[i]: + self.launched[i] = True + self.counter.launched += 1 + self.finished[i] = True + self.counter.finished += 1 + if self.counter.finished == self.counter.n: + self.sched.tasksetFinished(self) # cancel taskset + return + + hostname = self.id_retry_host[(task.id, num_try)] \ + if (task.id, num_try) in self.id_retry_host else task.host + + abort = (self.numFailures[index] >= MAX_TASK_FAILURES) + + if TaskEndReason.maybe_oom(reason): + self.counter.fail_oom += 1 + task.mem = min(task.mem * 2, MAX_TASK_MEMORY) + logger.info("task %s oom, enlarge memory limit to %d, origin %d", task.id, task.mem, task.rdd.mem) + + mem90 = self.mem_digest.quantile(0.9) + if not math.isnan(mem90): + mem90 = int(mem90) + if mem90 > self.mem90: + logger.info("enlarge memory limit of remaining task from >%d to >%d (mem90)", self.mem90, mem90) + self.mem90 = mem90 + for i, t in enumerate(self.tasks): + if not self.launched[i]: + t.mem = max(mem90, t.mem) + + else: + _logger = logger.error if abort else logger.warning + + err_msg_simple = '{} id: {}, host: {}, reason: {}'.format(status, task.id, hostname, reason) + err_msg = "{} message: {}".format(err_msg_simple, message) + + if status == TaskState.failed: + if reason is not None: # for tests with master=local + if reason.startswith("FATAL_EXCEPTION"): + self._abort('Job abort without retry for {}.'.format(reason)) + if reason not in self.reasons: + self.reasons.add(reason) + elif not abort: + err_msg = err_msg_simple + _logger(err_msg) + + self.counter.fail_all += 1 + self.numFailures[index] += 1 + + if abort: + self._abort('Task %s failed more than %d times' % (self.tasks[index].id, MAX_TASK_FAILURES)) + + task.reason_next = "fail" + + self.task_host_manager.task_failed(task.id, hostname, reason) + self.launched[index] = False + if self.counter.launched == self.counter.n: + self.sched.requestMoreResources() + self.running_hosts[index] = [] + self.counter.launched -= 1 + + def check_task_timeout(self): + """In lock, so be fast!""" + + now = time.time() + if self.last_check + 5 > now: + return False + self.last_check = now + + n = self.launched.count(True) + if n != self.counter.launched: + logger.warning( + 'bug: counter.launched(%d) != %d', + self.counter.launched, + n) + self.counter.launched = n + + # staged but not run for too long + # mesos may be busy. + num_resubmit = 0 + for i in range(self.counter.n): + task = self.tasks[i] + if (self.launched[i] and task.status == TaskState.staging + and task.stage_time + self.max_stage_time + WAIT_FOR_RUNNING < now): + logger.warning('task %s staging timeout %.1f (at %s), re-assign it', + task.id, now - task.stage_time, task.host) + self.counter.fail_staging_timeout += 1 + task.reason_next = TaskReason.stage_timeout + self.launched[i] = False + self.counter.launched -= 1 + num_resubmit += 1 + if num_resubmit > 3: + break + + # running for too long + num_resubmit = 0 + if self.counter.finished > self.counter.n * 0.8: + scale = 1.0 * self.counter.n / self.counter.finished + tasks = sorted((task.start_time, i, task) + for i, task in enumerate(self.tasks) + if self.launched[i] and not self.finished[i] and task.status == TaskState.running) + for _t, idx, task in tasks: + time_used = now - task.start_time + if time_used > self.max_task_time * (4 ** task.num_try) * scale: # num_try starts from 1 + # re-submit timeout task + self.counter.fail_run_timeout += 1 + if task.num_try <= MAX_TASK_FAILURES: + logger.info('re-submit task %s for run timeout %.1f, max finished = %d, try %d', + task.id, time_used, int(self.max_task_time), task.num_try) + task.time_used += time_used + task.stage_time = 0 + task.start_time = 0 + self.launched[idx] = False + self.counter.launched -= 1 + task.reason_next = TaskReason.run_timeout + else: + logger.error('task %s timeout, aborting taskset %s', + task, self.id) + self._abort('task %s timeout' % task) + else: + break + num_resubmit += 1 + if num_resubmit > 3: + break + return self.counter.launched < n + + def _abort(self, message): + logger.error('abort the taskset: %s', message) + tasks = ' '.join(str(i) for i in range(len(self.finished)) + if not self.finished[i]) + logger.error('not finished tasks: %s', tasks) + self.failed = True + self.causeOfFailure = message + self.sched.tasksetFinished(self) + self.sched.abort() diff --git a/dpark/tracker.py b/dpark/tracker.py index 0ba1aef3..95fd3819 100644 --- a/dpark/tracker.py +++ b/dpark/tracker.py @@ -1,56 +1,70 @@ +from __future__ import absolute_import import socket import zmq -import logging import time -from dpark.env import env -from dpark.util import spawn +from dpark.utils import spawn +from dpark.utils.log import get_logger + +logger = get_logger(__name__) -logger = logging.getLogger(__name__) class TrackerMessage(object): pass + class StopTrackerMessage(TrackerMessage): pass + class SetValueMessage(TrackerMessage): def __init__(self, key, value): self.key = key self.value = value + class AddItemMessage(TrackerMessage): def __init__(self, key, item): self.key = key self.item = item + class RemoveItemMessage(TrackerMessage): def __init__(self, key, item): self.key = key self.item = item + class GetValueMessage(TrackerMessage): def __init__(self, key): self.key = key + class TrackerServer(object): locs = {} + def __init__(self): self.addr = None self.thread = None + self.ctx = None def start(self): + if self.ctx is None: + self.ctx = zmq.Context() self.thread = spawn(self.run) while self.addr is None: time.sleep(0.01) def stop(self): - sock = env.ctx.socket(zmq.REQ) + sock = self.ctx.socket(zmq.REQ) sock.connect(self.addr) sock.send_pyobj(StopTrackerMessage()) confirm_msg = sock.recv_pyobj() sock.close() self.thread.join() + if self.ctx is not None: + self.ctx.destroy() + self.ctx = None return confirm_msg def get(self, key): @@ -73,13 +87,14 @@ def remove(self, key, item): self.locs[key].remove(item) def run(self): - locs = self.locs - sock = env.ctx.socket(zmq.REP) + sock = self.ctx.socket(zmq.REP) port = sock.bind_to_random_port("tcp://0.0.0.0") self.addr = "tcp://%s:%d" % (socket.gethostname(), port) logger.debug("TrackerServer started at %s", self.addr) - def reply(msg): - sock.send_pyobj(msg) + + def reply(msg_): + sock.send_pyobj(msg_) + while True: msg = sock.recv_pyobj() if isinstance(msg, SetValueMessage): @@ -102,16 +117,27 @@ def reply(msg): sock.close() logger.debug("stop TrackerServer %s", self.addr) + class TrackerClient(object): def __init__(self, addr): self.addr = addr + self.ctx = None def call(self, msg): + if self.ctx is None: + self.ctx = zmq.Context() + + sock = None try: - sock = env.ctx.socket(zmq.REQ) + sock = self.ctx.socket(zmq.REQ) sock.connect(self.addr) sock.send_pyobj(msg) return sock.recv_pyobj() finally: - sock.close() + if sock: + sock.close() + def stop(self): + if self.ctx is not None: + self.ctx.destroy() + self.ctx = None diff --git a/dpark/util.py b/dpark/util.py deleted file mode 100644 index 4f937cf7..00000000 --- a/dpark/util.py +++ /dev/null @@ -1,60 +0,0 @@ -# util -import types -from zlib import compress as _compress, decompress -import threading -import warnings -try: - from dpark.portable_hash import portable_hash as _hash -except ImportError: - import pyximport - pyximport.install(inplace=True) - from dpark.portable_hash import portable_hash as _hash - -try: - import os - import pwd - def getuser(): - return pwd.getpwuid(os.getuid()).pw_name -except: - import getpass - def getuser(): - return getpass.getuser() - -COMPRESS = 'zlib' -def compress(s): - return _compress(s, 1) - -try: - from lz4 import compress, decompress - COMPRESS = 'lz4' -except ImportError: - try: - from snappy import compress, decompress - COMPRESS = 'snappy' - except ImportError: - pass - -def spawn(target, *args, **kw): - t = threading.Thread(target=target, name=target.__name__, args=args, kwargs=kw) - t.daemon = True - t.start() - return t - -# hash(None) is id(None), different from machines -# http://effbot.org/zone/python-hash.htm -def portable_hash(value): - return _hash(value) - -# similar to itertools.chain.from_iterable, but faster in PyPy -def chain(it): - for v in it: - for vv in v: - yield vv - -def izip(*its): - its = [iter(it) for it in its] - try: - while True: - yield tuple([it.next() for it in its]) - except StopIteration: - pass diff --git a/dpark/utils/__init__.py b/dpark/utils/__init__.py new file mode 100644 index 00000000..5bce03a9 --- /dev/null +++ b/dpark/utils/__init__.py @@ -0,0 +1,217 @@ +# utils +from __future__ import absolute_import +import os +import sys +import threading +import errno +import uuid +import time +import platform +import tempfile +import os.path +from contextlib import contextmanager +from zlib import compress as _compress +from dpark.utils.crc32c import crc32c + +try: + from dpark.portable_hash import portable_hash as _hash +except ImportError: + import pyximport + + pyximport.install(inplace=True) + from dpark.portable_hash import portable_hash as _hash + +try: + import pwd + + + def getuser(): + return pwd.getpwuid(os.getuid()).pw_name +except: + import getpass + + + def getuser(): + return getpass.getuser() + +COMPRESS = 'zlib' + + +class DparkUserFatalError(Exception): + pass + + +def compress(s): + return _compress(s, 1) + + +try: + from dpark.utils.lz4wrapper import compress, decompress + + COMPRESS = 'lz4' +except ImportError: + try: + from snappy import compress, decompress + + COMPRESS = 'snappy' + except ImportError: + pass + + +def spawn(target, *args, **kw): + t = threading.Thread(target=target, name=target.__name__, args=args, kwargs=kw) + t.daemon = True + t.start() + return t + + +# hash(None) is id(None), different from machines +# http://effbot.org/zone/python-hash.htm +def portable_hash(value): + return _hash(value) + + +# similar to itertools.chain.from_iterable, but faster in PyPy +def chain(it): + for v in it: + for vv in v: + yield vv + + +def izip(*its): + its = [iter(it) for it in its] + try: + while True: + yield tuple([next(it) for it in its]) + except StopIteration: + pass + + +def mkdir_p(path): + """like `mkdir -p`""" + try: + os.makedirs(path) + except OSError as exc: + if exc.errno == errno.EEXIST and os.path.isdir(path): + pass + else: + raise + + +def memory_str_to_mb(s): + lower = s.lower() + if lower[-1].isalpha(): + number, unit = float(lower[:-1]), lower[-1] + else: + number, unit = float(lower), 'm' + scale_factors = { + 'k': 1. / 1024, + 'm': 1, + 'g': 1024, + 't': 1024 * 1024, + } + return number * scale_factors[unit] + + +MIN_REMAIN_RECURSION_LIMIT = 150 + + +if platform.python_implementation() == 'PyPy': + MIN_REMAIN_RECURSION_LIMIT = 300 + def get_recursion_depth(): + import inspect + return len(inspect.stack()) +else: + from dpark.utils.recursion import get_recursion_depth + + +def recursion_limit_breaker(f): + def _(*a, **kw): + if get_recursion_depth() < sys.getrecursionlimit() - MIN_REMAIN_RECURSION_LIMIT: + return f(*a, **kw) + + def __(): + result = [] + finished = [] + cond = threading.Condition(threading.Lock()) + + def _run(): + it = iter(f(*a, **kw)) + with cond: + while True: + while result: + cond.wait() + try: + result.append(next(it)) + cond.notify() + except StopIteration: + break + + finished.append(1) + cond.notify() + + t = spawn(_run) + + with cond: + while True: + while not finished and not result: + cond.wait() + + if result: + yield result.pop() + cond.notify() + + if finished: + assert not result + break + + t.join() + + return __() + + return _ + + +class AbortFileReplacement(Exception): + pass + + +@contextmanager +def atomic_file(filename, mode='w+b', bufsize=-1): + path, name = os.path.split(filename) + path = path or None + prefix = '.%s.' % (name,) if name else '.' + suffix = '.%s.tmp' % (uuid.uuid4().hex,) + tempname = None + try: + try: + mkdir_p(path) + except (IOError, OSError): + time.sleep(1) # there are dir cache in mfs for 1 sec + mkdir_p(path) + + with tempfile.NamedTemporaryFile( + mode=mode, suffix=suffix, prefix=prefix, + dir=path, delete=False) as f: + tempname = f.name + yield f + + os.chmod(tempname, 0o644) + os.rename(tempname, filename) + except AbortFileReplacement: + pass + finally: + try: + if tempname: + os.remove(tempname) + except OSError: + pass + + +def masked_crc32c(s): + crc = crc32c(s) + return (((crc >> 15) | (crc << 17)) + 0xa282ead8) & 0xffffffff + + +def sec2nanosec(t): + return t * 10**9 diff --git a/dpark/utils/beansdb.py b/dpark/utils/beansdb.py new file mode 100644 index 00000000..51462c9b --- /dev/null +++ b/dpark/utils/beansdb.py @@ -0,0 +1,386 @@ +from __future__ import absolute_import +from __future__ import print_function +import marshal +import binascii +import os +import socket +import time +import struct +import zlib +from dpark.utils.log import get_logger +from dpark.file_manager import open_file +from dpark.serialize import load_func, dump_func +from contextlib import closing +import six +from six.moves import range, cPickle + +logger = get_logger(__name__) +try: + import quicklz +except ImportError: + quicklz = None + +try: + from fnv1a import get_hash + from fnv1a import get_hash_beansdb + + + def fnv1a(d): + return get_hash(d) & 0xffffffff + + + def fnv1a_beansdb(d): + return get_hash_beansdb(d) & 0xffffffff +except ImportError: + FNV_32_PRIME = 0x01000193 + FNV_32_INIT = 0x811c9dc5 + + + def fnv1a(d): + h = FNV_32_INIT + for c in six.iterbytes(d): + h ^= c + h *= FNV_32_PRIME + h &= 0xffffffff + return h + + + fnv1a_beansdb = fnv1a + +FLAG_PICKLE = 0x00000001 +FLAG_INTEGER = 0x00000002 +FLAG_LONG = 0x00000004 +FLAG_BOOL = 0x00000008 +FLAG_COMPRESS1 = 0x00000010 # by cmemcached +FLAG_MARSHAL = 0x00000020 +FLAG_COMPRESS = 0x00010000 # by beansdb + +PADDING = 256 +BEANSDB_MAX_KEY_LENGTH = 250 +BEANSDB_MAX_VALUE_LENGTH = 500 << 20 + + +def check_size(ksz, vsz): + if not (0 < ksz <= BEANSDB_MAX_KEY_LENGTH and 0 <= vsz <= BEANSDB_MAX_VALUE_LENGTH): + return 'bad key/value size len(key)={} len(value)={}'.format(ksz, vsz) + + +def is_valid_key(key): + if len(key) > BEANSDB_MAX_KEY_LENGTH: + return False + invalid_chars = b' \r\n\0' + return not any(c in key for c in invalid_chars) + + +def restore_value(flag, val): + if flag & FLAG_COMPRESS: + val = quicklz.decompress(val) + if flag & FLAG_COMPRESS1: + val = zlib.decompress(val) + + if flag & FLAG_BOOL: + val = bool(int(val)) + elif flag & FLAG_INTEGER: + val = int(val) + elif flag & FLAG_MARSHAL: + val = marshal.loads(val) + elif flag & FLAG_PICKLE: + val = cPickle.loads(val) + return val + + +def prepare_value(val, compress): + flag = 0 + if isinstance(val, six.binary_type): + pass + elif isinstance(val, bool): + flag = FLAG_BOOL + val = str(int(val)).encode('utf-8') + elif isinstance(val, six.integer_types): + flag = FLAG_INTEGER + val = str(val).encode('utf-8') + elif isinstance(val, six.text_type): + flag = FLAG_MARSHAL + val = marshal.dumps(val, 2) + else: + try: + val = marshal.dumps(val, 2) + flag = FLAG_MARSHAL + except ValueError: + val = cPickle.dumps(val, -1) + flag = FLAG_PICKLE + + if compress and len(val) > 1024: + flag |= FLAG_COMPRESS + val = quicklz.compress(val) + + return flag, val + + +def read_record(f, check_crc=False): + block = f.read(PADDING) + if len(block) < 24: # + return None, "EOF" + crc, tstamp, flag, ver, ksz, vsz = struct.unpack("IiiiII", block[:24]) + err = check_size(ksz, vsz) + if err: + return None, err + + rsize = 24 + ksz + vsz + if rsize & 0xff: + rsize = ((rsize >> 8) + 1) << 8 + if rsize > PADDING: + n = rsize - PADDING + remain = f.read(n) + if len(remain) != n: + return None, "EOF data" + block += remain + if check_crc: + crc32 = binascii.crc32(block[4:24 + ksz + vsz]) & 0xffffffff + if crc32 != crc: + return None, "crc wrong" + key = block[24:24 + ksz] + value = block[24 + ksz:24 + ksz + vsz] + return (rsize, key, ((flag, value), ver, tstamp)), None + + +def write_record(f, key, flag, value, version, ts): + err = check_size(len(key), len(value)) + if err: + raise Exception(err) + + header = struct.pack('IIiII', ts, flag, version, len(key), len(value)) + crc32 = binascii.crc32(header) + crc32 = binascii.crc32(key, crc32) + crc32 = binascii.crc32(value, crc32) & 0xffffffff + f.write(struct.pack("I", crc32)) + f.write(header) + f.write(key) + f.write(value) + rsize = 24 + len(key) + len(value) + if rsize & 0xff: + f.write(b'\x00' * (PADDING - (rsize & 0xff))) + rsize = ((rsize >> 8) + 1) << 8 + return rsize + + +class BeansdbReader(object): + + def __init__(self, path, key_filter=None, fullscan=False, raw=False): + if key_filter is None: + fullscan = True + self.path = path + self.key_filter = key_filter + self.fullscan = fullscan + self.raw = raw + if not fullscan: + hint = path[:-5] + '.hint' + if not os.path.exists(hint) and not os.path.exists(hint + '.qlz'): + self.fullscan = True + + def __getstate__(self): + d = dict(self.__dict__) + del d['key_filter'] + return d, dump_func(self.key_filter) + + def __setstate__(self, state): + self.__dict__, code = state + try: + self.key_filter = load_func(code) + except Exception: + raise + + def read(self, begin, end): + if self.fullscan: + return self.full_scan(begin, end) + hint = self.path[:-5] + '.hint.qlz' + if os.path.exists(hint): + return self.scan_hint(hint) + hint = self.path[:-5] + '.hint' + if os.path.exists(hint): + return self.scan_hint(hint) + return self.full_scan(begin, end) + + def scan_hint(self, hint_path): + with open(hint_path, 'rb') as f: + hint = f.read() + + if hint_path.endswith('.qlz'): + try: + hint = quicklz.decompress(hint) + except ValueError as e: + msg = str(e) + if msg.startswith('compressed length not match'): + hint = hint[:int(msg.split('!=')[1])] + hint = quicklz.decompress(hint) + + key_filter = self.key_filter or (lambda x: True) + with open(self.path, 'rb') as dataf: + p = 0 + while p < len(hint): + pos, ver, _ = struct.unpack("IiH", hint[p:p + 10]) + p += 10 + ksz = pos & 0xff + key = hint[p: p + ksz] + if key_filter(key): + dataf.seek(pos & 0xffffff00) + r, err = read_record(dataf) + if err is not None: + logger.error("read failed from %s at %d", + self.path, pos & 0xffffff00) + else: + rsize, key, value = r + value, err = self.restore(value) + if not err: + yield key, value + p += ksz + 1 # \x00 + + def restore(self, value): + err = None + if not self.raw: + try: + value = restore_value(*value) + except Exception as e: + err = "restore expection: %s value %s" % (e, value) + logger.error(err) + return value, err + + def open_file(self): + return open_file(self.path) + + def full_scan(self, begin, end): + with closing(self.open_file()) as f: + # try to find first record + while True: + f.seek(begin) + r, err = read_record(f, check_crc=True) + if err is None: + break + begin += PADDING + if begin >= end: + break + if begin >= end: + return + + f.seek(begin) + key_filter = self.key_filter or (lambda x: True) + while begin < end: + r, err = read_record(f) + if err: + logger.error('read error at %s pos: %d err: %s', + self.path, begin, err) + if err == "EOF": + return + begin += PADDING + while begin < end: + f.seek(begin) + r, err = read_record(f, check_crc=True) + if err is not None: + break + begin += PADDING + continue + size, key, value = r + if key_filter(key): + value, err = self.restore(value) + if not err: + yield key, value + begin += size + + +class BeansdbWriter(object): + + def __init__(self, path, depth, overwrite, compress=False, + raw=False, value_with_meta=False): + self.path = path + self.depth = depth + self.overwrite = overwrite + if not quicklz: + compress = False + self.compress = compress + self.raw = raw + self.value_with_meta = value_with_meta + for i in range(16 ** depth): + if depth > 0: + ps = list(('%%0%dx' % depth) % i) + p = os.path.join(path, *ps) + else: + p = path + if os.path.exists(p): + if overwrite: + for n in os.listdir(p): + if n[:3].isdigit(): + os.remove(os.path.join(p, n)) + else: + os.makedirs(p) + + def prepare(self, val): + if self.raw: + return val + + return prepare_value(val, self.compress) + + def write_record(self, f, key, value, now): + if self.value_with_meta: + value, version, ts = value + else: + version = 1 + ts = now + if self.raw: + flag, value = value + else: + flag, value = self.prepare(value) + return write_record(f, key, flag, value, version, ts) + + def write_bucket(self, it, index): + """ 0 <= index < 256 + yield from it + write to "*/%03d.data" % index + """ + N = 16 ** self.depth + if self.depth > 0: + fmt = '%%0%dx' % self.depth + ds = [os.path.join(self.path, *list(fmt % i)) for i in range(N)] + else: + ds = [self.path] + pname = '%03d.data' % index + tname = '.%03d.data.%s.tmp' % (index, socket.gethostname()) + p = [os.path.join(d, pname) for d in ds] + tp = [os.path.join(d, tname) for d in ds] + pos = [0] * N + f = [open(t, 'wb', 1 << 20) for t in tp] + now = int(time.time()) + hint = [[] for d in ds] + + bits = 32 - self.depth * 4 + for key, value in it: + if not isinstance(key, (six.string_types, six.binary_type)): + key = str(key) + + if isinstance(key, six.text_type): + key = key.encode('utf-8') + + if not is_valid_key(key): + logger.warning("ignored invalid key: %s", [key]) + continue + + i = fnv1a(key) >> bits + + hint[i].append(struct.pack("IIH", pos[i] + len(key), 1, 0) + key + b'\x00') + pos[i] += self.write_record(f[i], key, value, now) + for i in f: + i.close() + + for i in range(N): + if hint[i] and not os.path.exists(p[i]): + os.rename(tp[i], p[i]) + hintdata = b''.join(hint[i]) + hint_path = os.path.join(os.path.dirname(p[i]), '%03d.hint' % index) + if self.compress: + hintdata = quicklz.compress(hintdata) + hint_path += '.qlz' + with open(hint_path, 'wb') as f: + f.write(hintdata) + else: + os.remove(tp[i]) + + return sum([([p[i]] if hint[i] else []) for i in range(N)], []) diff --git a/dpark/bitindex.py b/dpark/utils/bitindex.py similarity index 79% rename from dpark/bitindex.py rename to dpark/utils/bitindex.py index a143919b..6a8cd820 100644 --- a/dpark/bitindex.py +++ b/dpark/utils/bitindex.py @@ -1,14 +1,19 @@ -import itertools +from __future__ import absolute_import import marshal import math from dpark.portable_hash import portable_hash +from six.moves import range +from functools import reduce +from six.moves import zip as izip +from six.moves import zip_longest as izip_longest BYTE_SHIFT = 3 BYTE_SIZE = 1 << BYTE_SHIFT BYTE_MASK = BYTE_SIZE - 1 -_table = [(), (0,), (1,), (0,1), (2,), (0,2), (1,2), (0,1,2), (3,), - (0,3), (1,3), (0,1,3), (2,3), (0,2,3), (1,2,3), (0,1,2,3)] +_table = [(), (0,), (1,), (0, 1), (2,), (0, 2), (1, 2), (0, 1, 2), (3,), + (0, 3), (1, 3), (0, 1, 3), (2, 3), (0, 2, 3), (1, 2, 3), (0, 1, 2, 3)] + class BitIndex(object): def __init__(self): @@ -52,7 +57,7 @@ def _bitwise(iterable, op): for x in _table[byte >> 4]: yield i * BYTE_SIZE + 4 + x - def set(self, pos, value = True): + def set(self, pos, value=True): if pos < 0: raise ValueError('pos must great or equal zero!') @@ -68,11 +73,11 @@ def set(self, pos, value = True): self.size = max(self.size, pos + 1) - def sets(self, positions, value = True): + def sets(self, positions, value=True): for pos in positions: self.set(pos, value) - def append(self, value = True): + def append(self, value=True): self.set(self.size, value) def appends(self, values): @@ -95,25 +100,26 @@ def gets(self, positions): def intersect(self, *other): return self._bitwise( - itertools.izip(self.array, *[o.array for o in other]), lambda x, y: x & y) + izip(self.array, *[o.array for o in other]), lambda x, y: x & y) def union(self, *other): return self._bitwise( - itertools.izip_longest(self.array, *[o.array for o in other], fillvalue=0), + izip_longest(self.array, *[o.array for o in other], fillvalue=0), lambda x, y: x | y) def xor(self, other): - return self._bitwise(itertools.izip_longest(self.array, other.array, fillvalue=0), - lambda x, y: x ^ y) + return self._bitwise(izip_longest(self.array, other.array, fillvalue=0), + lambda x, y: x ^ y) def excepts(self, *other): return self._bitwise( - itertools.izip_longest(self.array, *[o.array for o in other], fillvalue=0), + izip_longest(self.array, *[o.array for o in other], fillvalue=0), lambda x, y: x & ~y) def positions(self): return self._bitwise(self.array, None) + class Bloomfilter(object): def __init__(self, m, k): self.m = m @@ -129,22 +135,23 @@ def __init__(self, m, k): m = ceil((n * log(p)) / log(1.0 / (pow(2.0, log(2.0))))) k = round(log(2.0) * m / n) """ + @staticmethod def calculate_parameters(n, p): m = int(math.ceil(n * math.log(p) * -2.0813689810056073)) k = int(round(0.6931471805599453 * m / n)) return m, k - ''' we're using only two hash functions with different settings, as described - by Kirsch & Mitzenmacher: http://www.eecs.harvard.edu/~kirsch/pubs/bbbf/esa06.pdf + by Kirsch & Mitzenmacher: https://www.eecs.harvard.edu/~michaelm/postscripts/tr-02-05.pdf ''' + def _get_offsets(self, obj): hash_1 = portable_hash(obj) hash_2 = portable_hash(marshal.dumps(obj)) - for i in xrange(self.k): + for i in range(self.k): yield ((hash_1 + i * hash_2) & 0xFFFFFFFF) % self.m def add(self, objs): @@ -159,5 +166,4 @@ def match(self, objs): return list(self._match(objs)) def __contains__(self, obj): - return self._match([obj]).next() - + return next(self._match([obj])) diff --git a/dpark/utils/crc32c.c b/dpark/utils/crc32c.c new file mode 100644 index 00000000..ab30ecc5 --- /dev/null +++ b/dpark/utils/crc32c.c @@ -0,0 +1,312 @@ +/* crc32c.c -- compute CRC-32C using the Intel crc32 instruction + * Copyright (C) 2013 Mark Adler + * Version 1.1 1 Aug 2013 Mark Adler + */ + +/* + This software is provided 'as-is', without any express or implied + warranty. In no event will the author be held liable for any damages + arising from the use of this software. + + Permission is granted to anyone to use this software for any purpose, + including commercial applications, and to alter it and redistribute it + freely, subject to the following restrictions: + + 1. The origin of this software must not be misrepresented; you must not + claim that you wrote the original software. If you use this software + in a product, an acknowledgment in the product documentation would be + appreciated but is not required. + 2. Altered source versions must be plainly marked as such, and must not be + misrepresented as being the original software. + 3. This notice may not be removed or altered from any source distribution. + + Mark Adler + madler@alumni.caltech.edu + */ + +/* Use hardware CRC instruction on Intel SSE 4.2 processors. This computes a + CRC-32C, *not* the CRC-32 used by Ethernet and zip, gzip, etc. +*/ + +/* Version history: + 1.0 10 Feb 2013 First version + 1.1 1 Aug 2013 Correct comments on why three crc instructions in parallel + */ + +#include +#include +#include +#include +#include + +/* CRC-32C (iSCSI) polynomial in reversed bit order. */ +#define POLY 0x82f63b78 + +/* Table for a quadword-at-a-time software crc. */ +static pthread_once_t crc32c_once_sw = PTHREAD_ONCE_INIT; +static uint32_t crc32c_table[8][256]; + + +/* Construct table for software CRC-32C calculation. */ +static void crc32c_init_sw(void) +{ + uint32_t n, crc, k; + + for (n = 0; n < 256; n++) { + crc = n; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc = crc & 1 ? (crc >> 1) ^ POLY : crc >> 1; + crc32c_table[0][n] = crc; + } + for (n = 0; n < 256; n++) { + crc = crc32c_table[0][n]; + for (k = 1; k < 8; k++) { + crc = crc32c_table[0][crc & 0xff] ^ (crc >> 8); + crc32c_table[k][n] = crc; + } + } +} + +/* Table-driven software version as a fall-back. This is about 15 times slower + than using the hardware instructions. This assumes little-endian integers, + as is the case on Intel processors that the assembler code here is for. */ +uint32_t crc32c_sw(uint32_t crci, const void *buf, size_t len) +{ + const unsigned char *next = buf; + uint64_t crc; + + pthread_once(&crc32c_once_sw, crc32c_init_sw); + crc = crci ^ 0xffffffff; + while (len && ((uintptr_t)next & 7) != 0) { + crc = crc32c_table[0][(crc ^ *next++) & 0xff] ^ (crc >> 8); + len--; + } + while (len >= 8) { + crc ^= *(uint64_t *)next; + crc = crc32c_table[7][crc & 0xff] ^ + crc32c_table[6][(crc >> 8) & 0xff] ^ + crc32c_table[5][(crc >> 16) & 0xff] ^ + crc32c_table[4][(crc >> 24) & 0xff] ^ + crc32c_table[3][(crc >> 32) & 0xff] ^ + crc32c_table[2][(crc >> 40) & 0xff] ^ + crc32c_table[1][(crc >> 48) & 0xff] ^ + crc32c_table[0][crc >> 56]; + next += 8; + len -= 8; + } + while (len) { + crc = crc32c_table[0][(crc ^ *next++) & 0xff] ^ (crc >> 8); + len--; + } + return (uint32_t)crc ^ 0xffffffff; +} + +/* Block sizes for three-way parallel crc computation. LONG and SHORT must + both be powers of two. The associated string constants must be set + accordingly, for use in constructing the assembler instructions. */ +#define LONG 8192 +#define LONGx1 "8192" +#define LONGx2 "16384" +#define SHORT 256 +#define SHORTx1 "256" +#define SHORTx2 "512" + +/* Tables for hardware crc that shift a crc by LONG and SHORT zeros. */ +static pthread_once_t crc32c_once_hw = PTHREAD_ONCE_INIT; +static uint32_t crc32c_long[4][256]; +static uint32_t crc32c_short[4][256]; + +/* Multiply a matrix times a vector over the Galois field of two elements, + GF(2). Each element is a bit in an unsigned integer. mat must have at + least as many entries as the power of two for most significant one bit in + vec. */ +static inline uint32_t gf2_matrix_times(uint32_t *mat, uint32_t vec) +{ + uint32_t sum; + + sum = 0; + while (vec) { + if (vec & 1) + sum ^= *mat; + vec >>= 1; + mat++; + } + return sum; +} + +/* Multiply a matrix by itself over GF(2). Both mat and square must have 32 + rows. */ +static inline void gf2_matrix_square(uint32_t *square, uint32_t *mat) +{ + int n; + + for (n = 0; n < 32; n++) + square[n] = gf2_matrix_times(mat, mat[n]); +} + +/* Construct an operator to apply len zeros to a crc. len must be a power of + two. If len is not a power of two, then the result is the same as for the + largest power of two less than len. The result for len == 0 is the same as + for len == 1. A version of this routine could be easily written for any + len, but that is not needed for this application. */ +static void crc32c_zeros_op(uint32_t *even, size_t len) +{ + int n; + uint32_t row; + uint32_t odd[32]; /* odd-power-of-two zeros operator */ + + /* put operator for one zero bit in odd */ + odd[0] = POLY; /* CRC-32C polynomial */ + row = 1; + for (n = 1; n < 32; n++) { + odd[n] = row; + row <<= 1; + } + + /* put operator for two zero bits in even */ + gf2_matrix_square(even, odd); + + /* put operator for four zero bits in odd */ + gf2_matrix_square(odd, even); + + /* first square will put the operator for one zero byte (eight zero bits), + in even -- next square puts operator for two zero bytes in odd, and so + on, until len has been rotated down to zero */ + do { + gf2_matrix_square(even, odd); + len >>= 1; + if (len == 0) + return; + gf2_matrix_square(odd, even); + len >>= 1; + } while (len); + + /* answer ended up in odd -- copy to even */ + for (n = 0; n < 32; n++) + even[n] = odd[n]; +} + +/* Take a length and build four lookup tables for applying the zeros operator + for that length, byte-by-byte on the operand. */ +static void crc32c_zeros(uint32_t zeros[][256], size_t len) +{ + uint32_t n; + uint32_t op[32]; + + crc32c_zeros_op(op, len); + for (n = 0; n < 256; n++) { + zeros[0][n] = gf2_matrix_times(op, n); + zeros[1][n] = gf2_matrix_times(op, n << 8); + zeros[2][n] = gf2_matrix_times(op, n << 16); + zeros[3][n] = gf2_matrix_times(op, n << 24); + } +} + +/* Apply the zeros operator table to crc. */ +static inline uint32_t crc32c_shift(uint32_t zeros[][256], uint32_t crc) +{ + return zeros[0][crc & 0xff] ^ zeros[1][(crc >> 8) & 0xff] ^ + zeros[2][(crc >> 16) & 0xff] ^ zeros[3][crc >> 24]; +} + +/* Initialize tables for shifting crcs. */ +static void crc32c_init_hw(void) +{ + crc32c_zeros(crc32c_long, LONG); + crc32c_zeros(crc32c_short, SHORT); +} + +/* Compute CRC-32C using the Intel hardware instruction. */ +uint32_t crc32c(const void *buf, size_t len, uint32_t crc) +{ + const unsigned char *next = buf; + const unsigned char *end; + uint64_t crc0, crc1, crc2; /* need to be 64 bits for crc32q */ + + /* populate shift tables the first time through */ + pthread_once(&crc32c_once_hw, crc32c_init_hw); + + /* pre-process the crc */ + crc0 = crc ^ 0xffffffff; + + /* compute the crc for up to seven leading bytes to bring the data pointer + to an eight-byte boundary */ + while (len && ((uintptr_t)next & 7) != 0) { + __asm__("crc32b\t" "(%1), %0" + : "=r"(crc0) + : "r"(next), "0"(crc0)); + next++; + len--; + } + + /* compute the crc on sets of LONG*3 bytes, executing three independent crc + instructions, each on LONG bytes -- this is optimized for the Nehalem, + Westmere, Sandy Bridge, and Ivy Bridge architectures, which have a + throughput of one crc per cycle, but a latency of three cycles */ + while (len >= LONG*3) { + crc1 = 0; + crc2 = 0; + end = next + LONG; + do { + __asm__("crc32q\t" "(%3), %0\n\t" + "crc32q\t" LONGx1 "(%3), %1\n\t" + "crc32q\t" LONGx2 "(%3), %2" + : "=r"(crc0), "=r"(crc1), "=r"(crc2) + : "r"(next), "0"(crc0), "1"(crc1), "2"(crc2)); + next += 8; + } while (next < end); + crc0 = crc32c_shift(crc32c_long, crc0) ^ crc1; + crc0 = crc32c_shift(crc32c_long, crc0) ^ crc2; + next += LONG*2; + len -= LONG*3; + } + + /* do the same thing, but now on SHORT*3 blocks for the remaining data less + than a LONG*3 block */ + while (len >= SHORT*3) { + crc1 = 0; + crc2 = 0; + end = next + SHORT; + do { + __asm__("crc32q\t" "(%3), %0\n\t" + "crc32q\t" SHORTx1 "(%3), %1\n\t" + "crc32q\t" SHORTx2 "(%3), %2" + : "=r"(crc0), "=r"(crc1), "=r"(crc2) + : "r"(next), "0"(crc0), "1"(crc1), "2"(crc2)); + next += 8; + } while (next < end); + crc0 = crc32c_shift(crc32c_short, crc0) ^ crc1; + crc0 = crc32c_shift(crc32c_short, crc0) ^ crc2; + next += SHORT*2; + len -= SHORT*3; + } + + /* compute the crc on the remaining eight-byte units less than a SHORT*3 + block */ + end = next + (len - (len & 7)); + while (next < end) { + __asm__("crc32q\t" "(%1), %0" + : "=r"(crc0) + : "r"(next), "0"(crc0)); + next += 8; + } + len &= 7; + + /* compute the crc for up to seven trailing bytes */ + while (len) { + __asm__("crc32b\t" "(%1), %0" + : "=r"(crc0) + : "r"(next), "0"(crc0)); + next++; + len--; + } + + /* return a post-processed crc */ + return (uint32_t)(crc0 ^ 0xffffffff); +} diff --git a/dpark/utils/crc32c_mod.c b/dpark/utils/crc32c_mod.c new file mode 100644 index 00000000..235b5d0d --- /dev/null +++ b/dpark/utils/crc32c_mod.c @@ -0,0 +1,84 @@ +#include +#include + +uint32_t crc32c(const void *buf, size_t len, uint32_t crc); +uint32_t crc32c_sw(uint32_t crci, const void *buf, size_t len); + +PyDoc_STRVAR(doc_crc32c, +"(bytes, oldcrc = 0) -> newcrc. Compute CRC-32c incrementally"); + +static PyObject * +crc32c_crc32c(PyObject *self, PyObject *args) +{ + Py_buffer pbin; + unsigned char *bin_data; + unsigned int crc = 0U; /* initial value of CRC */ + Py_ssize_t len; + unsigned int result; + + if ( !PyArg_ParseTuple(args, "s*|I:crc32", &pbin, &crc) ) + return NULL; + + bin_data = pbin.buf; + len = pbin.len; + + if (__builtin_cpu_supports("sse4.2")) { + result = crc32c(bin_data, len, crc); + } else { + result = crc32c_sw(crc, bin_data, len); + } + + PyBuffer_Release(&pbin); + return PyLong_FromLong(result); +} + +static PyMethodDef crc32c_module_methods[] = { + {"crc32c", crc32c_crc32c, METH_VARARGS, doc_crc32c}, + {NULL, NULL} /* sentinel */ +}; + +PyDoc_STRVAR(doc_crc32c_module, +"Compute CRC32c with sse4.2 support"); + +#if PY_MAJOR_VERSION >= 3 + +static struct PyModuleDef crc32c_module_def = { + PyModuleDef_HEAD_INIT, + "crc32c", /* name of module */ + doc_crc32c_module, /* module documentation, may be NULL */ + -1, /* size of per-interpreter state of the module, + or -1 if the module keeps state in global variables. */ + crc32c_module_methods +}; + +PyObject * +PyInit_crc32c(void) { +#ifndef __clang__ + __builtin_cpu_init (); +#endif + PyObject *module = PyModule_Create(&crc32c_module_def); + if (module == NULL) + return NULL; + + return module; +} + +#else + +void +initcrc32c(void) { +#ifndef __clang__ + __builtin_cpu_init (); +#endif + PyObject *module, *d, *x; + module = Py_InitModule("crc32c", crc32c_module_methods); + if (module == NULL) + return; + + d = PyModule_GetDict(module); + x = PyString_FromString(doc_crc32c_module); + PyDict_SetItemString(d, "__doc__", x); + Py_XDECREF(x); +} + +#endif diff --git a/dpark/utils/dag.py b/dpark/utils/dag.py new file mode 100644 index 00000000..58af3a71 --- /dev/null +++ b/dpark/utils/dag.py @@ -0,0 +1,167 @@ +""" +TODO: + +- show size/time info for input/output/shuffle. +- show broadcast cache, checkpoint. +- show src file content on click (in a new tab). +- Try to get failed job/stage dag. +- Make struct of .prof more stable, decouple it from view. + +""" + +import os.path +import glob +import json +import time + + +KW_NODES = "nodes" +KW_ID = "id" # uniq, use in edges +KW_LABEL = "label" # short name +KW_DETAIL = "detail" # show when hover +KW_TYPE = "type" + +KW_EDGES = "edges" +KW_SRC = "source" +KW_DST = "target" + + +def from_loghub(path): + files = glob.glob(os.path.join(path, "sched*")) + jsons = [] + for p in files: + with open(p) as f: + jsons.append(json.load(f)) + + return trans(jsons) + + +def fmt_duraion(s): + i = int(s) + r = "" + for a, b in [(86400, 'd'), (3600, 'h'), (60, 'm')]: + if i >= a: + r += ("{}" + b).format(i / a) + i %= a + if r == "": + # return "{:.1e}s".format(s) + return "{}s".format(i + 1) + else: + return r + + +def M(b): + return int(b/(1024*1024)) + + +def summary_prof(p): + counters = p['counters'] + stats = p['stats'] + info = p['info'] + + task = counters['task'] + fail = counters['fail'] + + task_torun = task['all'] - task['running'] - task['finished'] + other_error = fail['all'] - fail['oom'] - fail['run_timeout'] - fail['staging_timeout'] - fail['fetch'] + finished = task['finished'] + res = [ + ["task", "{} = {} + {} + {}".format(task['all'], task['finished'], task['running'], task_torun)], + ["fail", "{} = {} + {} + {} + {} + {} ".format(fail['all'], + fail['oom'], + fail['fetch'], + fail['run_timeout'], + fail['staging_timeout'], + other_error + )], + ] + + if not stats: + return res + + mem = stats['bytes_max_rss'] + t = stats['secs_all'] + + end_time = info['finish_time'] + if end_time <= 0: + end_time = time.time() + + stage_time = end_time - info['start_time'] + if finished > 0: + avg_mem = mem['sum'] / finished + avg_time = t['sum'] / finished + speedup = avg_time * finished / stage_time + else: + avg_mem = 0 + avg_time = 0 + speedup = 0 + + res2 = [ + ['mem', "{} || [{}, {}, {}]".format(info['mem'], M(mem['min']), M(avg_mem), M(mem['max']))], + ['time', "{} || [{}, {}, {}]".format(*[fmt_duraion(s) for s in [stage_time, t['min'], avg_time, t['max']]])], + ['speedup', "{:.2f}".format(speedup)] + ] + return res + res2 + + +def trans(runs): + api_nodes = {} + api_edges = {} + stage_nodes = {} + stage_edges = {} + for r in runs: + r = r["run"] + for s in r['stages']: + for n in s['graph'][KW_NODES]: + if n[KW_ID] in stage_nodes: + continue + rdds = n['rdds'] + n['rdds'] = list(reversed([{"k": rdd["rdd_name"], + "v": str(rdd["api_callsite_id"]), + "params": rdd['params']} + for rdd in rdds])) + if n[KW_ID] == s['info']['output_pipeline']: + p = n['prof'] = { + 'info': s['info'], + 'stats': s['stats'], + 'counters': s['counters'] + } + n['prof_summary'] = summary_prof(p) + n['is_output'] = True + else: + n['is_output'] = False + + stage_nodes[n[KW_ID]] = n + for e in s['graph'][KW_EDGES]: + id_ = e[KW_SRC], e[KW_DST] + if id_ not in stage_edges: + e['info'] = ",".join(["{}={}".format(str(k), str(v)) for k, v in e["info"].items()]) + stage_edges[id_] = e + sink_node = r["sink"]['node'] + sink_node['call_id'] = str(sink_node['call_id']) + stage_nodes[sink_node[KW_ID]] = sink_node + sink_edge = r['sink'][KW_EDGES] + stage_edges[(sink_edge[KW_SRC], sink_edge[KW_DST])] = sink_edge + + c = r['call_graph'] + for n in c[KW_NODES]: + id_ = n[KW_ID] = n['call_id'] = str(n[KW_ID]) + api_nodes[id_] = n + for e in c[KW_EDGES]: + s, r = str(e[KW_SRC]), str(e[KW_DST]) + if (s, r) not in api_edges: + e[KW_SRC] = s + e[KW_DST] = r + api_edges[(s, r)] = e + res = { + "stages": { + KW_NODES: list(stage_nodes.values()), + KW_EDGES: list(stage_edges.values()), + }, + "calls": { + KW_NODES: list(api_nodes.values()), + KW_EDGES: list(api_edges.values()), + } + } + + return res diff --git a/dpark/utils/debug.py b/dpark/utils/debug.py new file mode 100644 index 00000000..84e6e5a7 --- /dev/null +++ b/dpark/utils/debug.py @@ -0,0 +1,13 @@ +import logging + + +def spawn_rconsole(env): + rfoo_logger = logging.getLogger('rfoo') + rfoo_logger.disabled = 1 + try: + from rfoo.utils import rconsole + import rfoo._rfoo + rfoo._rfoo.logging = rconsole.logging = rfoo_logger + rconsole.spawn_server(env, 0) + except ImportError: + pass diff --git a/dpark/utils/frame.py b/dpark/utils/frame.py new file mode 100644 index 00000000..4339e77d --- /dev/null +++ b/dpark/utils/frame.py @@ -0,0 +1,165 @@ +import os +import inspect +from collections import defaultdict +import linecache + + +def get_path(p): + return os.path.realpath(os.path.abspath(p)) + + +src_dir = os.path.dirname(os.path.dirname(get_path(__file__))) + + +class Frame(object): + + def __init__(self, f): + """working in func_name, exec code at pos""" + self.path = get_path(f.f_code.co_filename) + self.lineno = f.f_lineno + self.lasti = f.f_lasti + self.func_name = f.f_code.co_name + + @property + def pos(self): + return self.path, self.lineno, self.lasti + + +def frame_tuple(f): + return f.f_code.co_filename, f.f_lineno, f.f_lasti + + +def func_info(f): + co = getattr(f, "__code__", None) + if co: + return "{}@{}:{}".format(co.co_name, co.co_filename, co.co_firstlineno) + else: + return "{}".format(f) # builtin_function_or_method + + +def summary_stack(frames): + result = [] + for f in frames: + co = f.f_code + pos = '{}:{}, in {}'.format(co.co_filename, f.f_lineno, co.co_name) + line = linecache.getline(co.co_filename, f.f_lineno).strip() + if line: + line = line.strip() + # if f.f_locals: + # for name, value in sorted(f.f_locals.items()): + # row.append(' {name} = {value}\n'.format(name=name, value=value)) + result.append({"pos": pos, "line": line}) + return result + + +class Scope(object): + + scopes_by_id = {} + scopes_by_stackhash = {} + scopes_by_api_callsite_id = {} + + api_callsites = {} + calls_in_oneline = defaultdict(dict) # (path, line_no, fname) -> [lasti...] + gid = 0 + + def __init__(self, name_fmt, stack, stackhash, api, api_callsite, stack_above_api): + self.id = Scope.gid + Scope.gid += 1 + self.name = name_fmt.format(api=api) + self.stack = stack + self.stackhash = stackhash + self.api = api + self.api_callsite = api_callsite + self.key = "{}@{}".format(api, self.api_callsite) + self.api_callsite_id = self.api_callsites.get(api_callsite) + self.stack_above_api = stack_above_api + if self.api_callsite_id is None: + self.api_callsite_id = self.api_callsites[api_callsite] = len(self.api_callsites) + self.scopes_by_api_callsite_id[self.api_callsite_id] = [self] + else: + self.scopes_by_api_callsite_id[self.api_callsite_id].append(self) + # print(self.id, self.api_callsite_id, api_callsite, self.name) + + @classmethod + def reset(cls): + cls.scopes_by_id = {} + cls.scopes_by_stackhash = {} + cls.scopes_by_api_callsite_id = {} + + cls.api_callsites = {} + cls.calls_in_oneline = defaultdict(dict) + cls.gid = 0 + + @classmethod + def get_callsite(cls, caller, callee): + """ + Deal with usage like "rdd.map(_).map(_)", distinguish same dpark api called in one line by lasti. + To be comprehensible, replace lasti with order of calling of same api in this line , starts with 0. + """ + + callee = Frame(callee) # the dpark api called by user, DparkContext.xxx() or RDD.xxx() + caller = Frame(caller) # the first callsite out of dpark package, where user call dpark api + + key = caller.path, caller.lineno, callee.func_name + calls = cls.calls_in_oneline.setdefault(key, []) + i = -1 + for i, lasti in enumerate(calls): + if lasti == caller.lasti: + seq = i + break + else: + seq = i + 1 + calls.append(caller.lasti) + + api = callee.func_name + api_callsite = "{}:{}@{}:{}".format(callee.func_name, seq, caller.path, caller.lineno) + return api, api_callsite + + @classmethod + def get(cls, name_fmt): + callee = inspect.currentframe() + caller = callee.f_back + stack = [] + stack_above_api = [] + + api_caller = None + api_callee = None + + while caller is not None: + stack.append(frame_tuple(caller)) + if api_callee is None: + if src_dir != os.path.dirname(get_path(caller.f_code.co_filename)): + api_callee = callee # the dpark api called by user, DparkContext.xxx() or RDD.xxx() + api_caller = caller # the first callsite out of dpark package, where user call dpark api + stack_above_api.append(caller) + else: + stack_above_api.append(caller) + callee = caller + caller = caller.f_back + + stack = tuple(stack) + stackhash = hash(stack) + scope = cls.scopes_by_stackhash.get(stackhash) + if scope is None: + stack_above_api = summary_stack(stack_above_api) + api, api_callsite = cls.get_callsite(api_caller, api_callee) + scope = Scope(name_fmt, stack, stackhash, api, api_callsite, stack_above_api) + cls.scopes_by_stackhash[stackhash] = scope + cls.scopes_by_id[scope.id] = scope + return scope + + +def get_stacks_of_threads(): + import threading, sys, traceback + threads = {} + for t in threading.enumerate(): + f = sys._current_frames()[t.ident] + k = t.name + stack = traceback.format_stack() + v = { + "stack": stack, + "f_locals": "{}".format(f.f_locals), + "f_back.f_locals": "{}".format(f.f_back.f_locals) + } + threads[k] = v + return threads diff --git a/dpark/utils/heaponkey.py b/dpark/utils/heaponkey.py new file mode 100644 index 00000000..04c80011 --- /dev/null +++ b/dpark/utils/heaponkey.py @@ -0,0 +1,238 @@ +from __future__ import absolute_import +from __future__ import print_function +from dpark.serialize import load_func, dump_func +import sys +import operator +from six.moves import range + +if sys.version_info[0] < 3: + def next_func(it): + return it.next +else: + def next_func(it): + return it.__next__ + + +class HeapOnKey(object): + + def __init__(self, key=None, min_heap=False): + self.key = key + self.min_heap = min_heap + self._setup_cmp() + + def _setup_cmp(self): + key = self.key + min_heap = self.min_heap + + def _ge0(x, y): + return not (x < y) + + def _lt(x, y): + return key(x) < key(y) + + def _ge(x, y): + return not (key(x) < key(y)) + + if key is None: + self.cmp_lt = operator.lt if min_heap else _ge0 + else: + self.cmp_lt = _lt if min_heap else _ge + + def __getstate__(self): + return dump_func(self.key), self.min_heap + + def __setstate__(self, state): + key_f, self.min_heap = state + self.key = load_func(key_f) + self._setup_cmp() + + def push(self, heap, item): + heap.append(item) + self._sift_down(heap, 0, len(heap) - 1) + + def pop(self, heap): + last_item = heap.pop() + if heap: + ret_item = heap[0] + heap[0] = last_item + self._sift_up(heap, 0) + else: + ret_item = last_item + return ret_item + + def push_pop(self, heap, item): + if heap and self.cmp_lt(heap[0], item): + item, heap[0] = heap[0], item + self._sift_up(heap, 0) + return item + + def heapify(self, heap): + n = len(heap) + for i in range(n // 2 - 1, -1, -1): + self._sift_up(heap, i) + + def _sift_down(self, heap, start_pos, pos): + new_item = heap[pos] + cmp_lt = self.cmp_lt + while pos > start_pos: + parent_pos = (pos - 1) >> 1 + parent = heap[parent_pos] + if cmp_lt(new_item, parent): + heap[pos] = parent + pos = parent_pos + continue + break + heap[pos] = new_item + + def _sift_up(self, heap, pos): + end_pos = len(heap) + child_pos = 2 * pos + 1 + cmp_lt = self.cmp_lt + while child_pos < end_pos: + right_pos = child_pos + 1 + if right_pos < end_pos and not cmp_lt(heap[child_pos], heap[right_pos]): + child_pos = right_pos + if cmp_lt(heap[pos], heap[child_pos]): + break + heap[pos], heap[child_pos] = heap[child_pos], heap[pos] + pos = child_pos + child_pos = 2 * pos + 1 + + def replace(self, heap, item): + returnitem = heap[0] # raises appropriate IndexError if heap is empty + heap[0] = item + self._sift_up(heap, 0) + return returnitem + + def merge(self, iterables, ordered_iters=0): + """iterables: each is sorted + ordered_iters: when come to equal value, the element in the first iter yields + first(last) if ordered_iters >(<) 0 + not stable if ordered_iters == 0 + """ + if not ordered_iters: + + def key(x): + return self.key(x[0]) + else: + + def key(x): + return self.key(x[0]), x[1] + + heap = HeapOnKey(key, self.min_heap) + _heappop, _heapreplace, _StopIteration = heap.pop, heap.replace, StopIteration + _len = len + + h = [] + h_append = h.append + order = -1 if ordered_iters and ((ordered_iters > 0) ^ self.min_heap) else 1 + + for it_idx, it in enumerate(map(iter, iterables)): + try: + _next = next_func(it) + h_append([_next(), order * it_idx, _next]) + except _StopIteration: + pass + heap.heapify(h) + + while _len(h) > 1: + try: + while 1: + v, _, _next = s = h[0] + yield v + s[0] = _next() # raises StopIteration when exhausted + _heapreplace(h, s) # restore heap condition + except _StopIteration: + _heappop(h) # remove empty iterator + if h: + # fast case when only a single iterator remains + v, _, _next = h[0] + yield v + for v in _next.__self__: + yield v + + +def test(): + lst = [10, 9, 20, 18, 3, 24, 29, 39] + h = HeapOnKey() + h.heapify(lst) + import sys + print('the list after heapify:', lst, file=sys.stderr) + lst.pop() + lst[0] = 12 + h._sift_up(lst, 0) + print('the list after sift up:', lst, file=sys.stderr) + h.push(lst, 8) + print('the list after push:', lst, file=sys.stderr) + ret = h.pop(lst) + print('the list after pop:', lst, ' with value:', ret, file=sys.stderr) + h = HeapOnKey(min_heap=True) + h.heapify(lst) + h.push(lst, 12) + print('the list after reverse:', lst, file=sys.stderr) + + class Foo: + def __init__(self, name='', age=0, score=None): + self.name = name + self.age = age + self.score = score + + def __getstate__(self): + return self.name, self.age, self.score + + def __setstate__(self, state): + self.name, self.age, self.score = state + + def __repr__(self): + return '(name:' + self.name + ', age:' + str(self.age) + ', score(%d, %d, %d))' % self.score + + def key_func(foo): + return foo.age + + lst = [Foo('aaa', 10, (60, 89, 70)), Foo('bbb', 20, (78, 75, 60)), + Foo('ccc', 15, (60, 73, 84)), Foo('ddd', 21, (87, 64, 65)), + Foo('eee', 30, (54, 53, 79)), Foo('fff', 10, (87, 73, 98))] + h = HeapOnKey(key=key_func, min_heap=True) + h.heapify(lst) + print('the list after heapify:', lst, file=sys.stderr) + + len_l = 100000 + top_n = 10 + lst = [] + import random + for i in range(len_l): + lst.append(random.randint(1, 2 * len_l + 1)) + + top_l = [] + call_cnt = 0 + + def cnt_key(x): + global call_cnt + call_cnt += 1 + return x + + h = HeapOnKey(key=cnt_key) + import time + start = time.time() + for i in lst: + if len(top_l) >= top_n: + h.push_pop(top_l, i) + else: + h.push(top_l, i) + top_l.sort(key=cnt_key) + print('after heap:', top_l, ' with elapsed:', time.time() - start, ' with cnt:', call_cnt / 2, file=sys.stderr) + call_cnt = 0 + + start = time.time() + top_l = [] + for i in lst: + top_l.append(i) + top_l.sort(key=cnt_key) + if len(top_l) > top_n: + top_l.pop() + + print('after sort:', top_l, ' with elapsed:', time.time() - start, ' with cnt:', call_cnt / 2, file=sys.stderr) + + +if __name__ == '__main__': + test() diff --git a/dpark/hotcounter.py b/dpark/utils/hotcounter.py similarity index 61% rename from dpark/hotcounter.py rename to dpark/utils/hotcounter.py index 819143e4..d3ee410d 100644 --- a/dpark/hotcounter.py +++ b/dpark/utils/hotcounter.py @@ -1,7 +1,14 @@ +from __future__ import absolute_import +from __future__ import print_function import operator +import six +from six.moves import range + class HotCounter(object): - def __init__(self, vs=[], limit=20): + def __init__(self, vs=None, limit=20): + if vs is None: + vs = [] self.limit = limit self.total = {} self.updates = {} @@ -19,35 +26,40 @@ def add(self, v): self._merge() def _merge(self): - for k,c in self.updates.iteritems(): + for k, c in six.iteritems(self.updates): if c > 1: self.total[k] = self.total.get(k, 0) + c self._max = 0 self.updates = {} if len(self.total) > self.limit * 5: - self.total = dict(self.top(self.limit*3)) + self.total = dict(self.top(self.limit * 3)) def update(self, o): self._merge() if isinstance(o, HotCounter): o._merge() - for k,c in o.total.iteritems(): - self.total[k] = self.total.get(k,0) + c + for k, c in six.iteritems(o.total): + self.total[k] = self.total.get(k, 0) + c def top(self, limit): - return sorted(self.total.items(), key=operator.itemgetter(1), reverse=True)[:limit] + return sorted(list(self.total.items()), key=operator.itemgetter(1), reverse=True)[:limit] -if __name__ == '__main__': +def test(): import random import math + t = HotCounter() for j in range(10): c = HotCounter() - for i in xrange(10000): + for i in range(10000): v = int(math.sqrt(random.randint(0, 1000000))) c.add(v) t.update(c) - for k,v in t.top(20): - print k,v + for k, v in t.top(20): + print(k, v) + + +if __name__ == '__main__': + test() diff --git a/dpark/hyperloglog.py b/dpark/utils/hyperloglog.py similarity index 68% rename from dpark/hyperloglog.py rename to dpark/utils/hyperloglog.py index a17d5248..428b45ee 100644 --- a/dpark/hyperloglog.py +++ b/dpark/utils/hyperloglog.py @@ -1,42 +1,53 @@ +from __future__ import absolute_import +from __future__ import print_function import math from bisect import bisect_right import array +from six.moves import map +from six.moves import range +from six.moves import zip try: import pyhash + hash_func = pyhash.murmur2_x64_64a() HASH_LEN = 64 raise ImportError except ImportError: HASH_LEN = 30 + + def hash_func(v): return hash(v) & 0x3fffffff SPARSE = 0 NORMAL = 1 + class HyperLogLog(object): - def __init__(self, items=[], b=None, err=0.01): + def __init__(self, items=None, b=None, err=0.01): + if items is None: + items = [] assert 0.005 <= err < 0.14, 'must 0.005 < err < 0.14' if b is None: b = int(math.ceil(math.log((1.04 / err) ** 2, 2))) self.alpha = self._get_alpha(b) self.b = b self.m = 1 << b - self.M = None #array.array('B', [0] * self.m) + self.M = None # array.array('B', [0] * self.m) self.threshold = self.m / 20 self.items = set(items) - self.mask = (1< 0 else E - elif E <= float(1L << HASH_LEN) / 30.0: # intermidiate range correction -> No correction + elif E <= float(1 << HASH_LEN) / 30.0: # intermidiate range correction -> No correction return E else: - return -(1L << HASH_LEN) * math.log(1.0 - E / (1L << HASH_LEN)) + return -(1 << HASH_LEN) * math.log(1.0 - E / (1 << HASH_LEN)) + def test(l, err=0.03): - hll = HyperLogLog(err) + hll = HyperLogLog(err=err) for i in l: - hll.add(str(i)+'ip') + hll.add(str(i) + 'ip') le = len(hll) - print err*100.0, len(hll.M), len(l), le, (le-len(l)) * 100.0 / len(l) + print(err * 100.0, len(hll.M), len(l), le, (le - len(l)) * 100.0 / len(l)) + if __name__ == '__main__': for e in (0.005, 0.01, 0.03, 0.05, 0.1): - test(xrange(100), e) - test(xrange(10000), e) - test(xrange(100000), e) -# test(xrange(1000000), e) + test(range(100), e) + test(range(10000), e) + test(range(100000), e) +# test(range(1000000), e) diff --git a/dpark/utils/log.py b/dpark/utils/log.py new file mode 100644 index 00000000..d2541c90 --- /dev/null +++ b/dpark/utils/log.py @@ -0,0 +1,229 @@ +import os +import sys +import json +import logging +import logging.handlers +import re +from datetime import datetime + +LOG_FORMAT = '{GREEN}%(asctime)-15s{RESET}' \ + ' [%(levelname)s] [%(threadName)s] [%(name)-9s:%(lineno)d] %(message)s' +DATE_FORMAT = '%Y-%m-%d %H:%M:%S' + +RESET = "\033[0m" +BOLD = "\033[1m" +BLACK, RED, GREEN, YELLOW, BLUE, MAGENTA, CYAN, WHITE = [ + "\033[1;%dm" % i for i in range(30, 38) +] + +PALLETE = { + 'RESET': RESET, + 'BOLD': BOLD, + 'BLACK': BLACK, + 'RED': RED, + 'GREEN': GREEN, + 'YELLOW': YELLOW, + 'BLUE': BLUE, + 'MAGENTA': MAGENTA, + 'CYAN': CYAN, + 'WHITE': WHITE, +} + +COLORS = { + 'WARNING': YELLOW, + 'INFO': WHITE, + 'DEBUG': BLUE, + 'CRITICAL': YELLOW, + 'ERROR': RED +} + +FORMAT_PATTERN = re.compile('|'.join('{%s}' % k for k in PALLETE)) + + +def formatter_message(message, use_color=True): + if use_color: + return FORMAT_PATTERN.sub( + lambda m: PALLETE[m.group(0)[1:-1]], + message + ) + + return FORMAT_PATTERN.sub('', message) + + +class ColoredFormatter(logging.Formatter): + def __init__(self, fmt=None, datefmt=None, use_color=True): + if fmt: + fmt = formatter_message(fmt, use_color) + + logging.Formatter.__init__(self, fmt=fmt, datefmt=datefmt) + self.use_color = use_color + + def format(self, record): + record = logging.makeLogRecord(record.__dict__) + levelname = record.levelname + if self.use_color and levelname in COLORS: + levelname_color = COLORS[levelname] + levelname + RESET + record.levelname = levelname_color + + record.msg = formatter_message(record.msg, self.use_color) + return logging.Formatter.format(self, record) + + +USE_UTF8 = getattr(sys.stderr, 'encoding', None) == 'UTF-8' + +ASCII_BAR = ('[ ', ' ]', '#', '-', '-\\|/-\\|') +UNICODE_BAR = (u'[ ', u' ]', u'\u2589', u'-', + u'-\u258F\u258E\u258D\u258C\u258B\u258A') + + +def make_progress_bar(ratio, size=14): + if USE_UTF8: + L, R, B, E, F = UNICODE_BAR + else: + L, R, B, E, F = ASCII_BAR + + if size > 4: + n = size - 4 + with_border = True + else: + n = size + with_border = False + + p = n * ratio + blocks = int(p) + if p > blocks: + frac = int((p - blocks) * 7) + blanks = n - blocks - 1 + C = F[frac] + else: + blanks = n - blocks + C = '' + + if with_border: + return ''.join([L, B * blocks, C, E * blanks, R]) + else: + return ''.join([B * blocks, C, E * blanks]) + + +def init_dpark_logger(log_level, use_color=None): + logger = get_logger('dpark') + logger.propagate = False + + handler = logging.StreamHandler(stream=sys.stderr) + handler.setFormatter(ColoredFormatter(LOG_FORMAT, DATE_FORMAT, use_color)) + + handler.setLevel(max(log_level, logger.level)) + logger.addHandler(handler) + logger.setLevel(logging.DEBUG) + + +def get_logger(name): + """ Always use logging.Logger class. + + The user code may change the loggerClass (e.g. pyinotify), + and will cause exception when format log message. + """ + old_class = logging.getLoggerClass() + logging.setLoggerClass(logging.Logger) + logger = logging.getLogger(name) + logging.setLoggerClass(old_class) + return logger + + +def add_loghub(framework_id): + logger = get_logger('dpark') + try: + import dpark + from dpark.conf import LOGHUB, ENABLE_ES_LOGHUB, ES_HOST, ES_INDEX, ES_TYPE, LOGHUB_PATH_FORMAT + from dpark.utils import getuser + date_str = datetime.now().strftime(LOGHUB_PATH_FORMAT) + date_dir_path = os.path.join(LOGHUB, date_str) + if not os.path.exists(date_dir_path): + logger.error("loghub dir not ready: %s", date_dir_path) + return + + dir_path = os.path.join(date_dir_path, framework_id) + os.mkdir(dir_path) + + infos = [ + ("CMD", ' '.join(sys.argv)), + ("USER", getuser()), + ("PWD", os.getcwd()), + ("CTIME", datetime.strftime(datetime.now(), "%Y-%m-%d %H:%M:%S")), + ("DPARK", dpark.__file__), + ("PYTHONPATH", os.environ.get("PYTHONPATH", "")) + ] + + log_path = os.path.join(dir_path, "log") + try: + with open(log_path, "a") as f: + for i in infos: + f.write("DPARK_{} = {}\n".format(i[0], i[1])) + f.write("\n") + except IOError: + logger.exception("fail to write loghub: %s", log_path) + return + + if ENABLE_ES_LOGHUB: + es_handler = ElasticSearchHandler(ES_HOST, ES_INDEX, ES_TYPE, + infos, log_path) + es_handler.setLevel(logging.WARNING) + logger.addHandler(es_handler) + + file_handler = logging.FileHandler(filename=log_path) + file_handler.setFormatter(ColoredFormatter(LOG_FORMAT, DATE_FORMAT, True)) + file_handler.setLevel(logging.INFO) + logger.addHandler(file_handler) + logger.info("logging/prof to %s", dir_path) + return file_handler, dir_path + except Exception: + logger.exception("add_loghub fail") + + +def create_logger(stream, handler=None): + logger = get_logger('dpark.' + str(stream.fileno())) + logger.propagate = False + stream_handler = logging.StreamHandler(stream=stream) + stream_handler.setFormatter(logging.Formatter()) + stream_handler.setLevel(logging.INFO) + logger.addHandler(stream_handler) + if handler: + logger.addHandler(handler) + + return logger + + +class ElasticSearchHandler(logging.handlers.HTTPHandler): + + def __init__(self, host, base_index, _type, infos, loghub_file, timeout=2): + logging.Handler.__init__(self) + self.host = host + self.url = '/{}-{:%Y-%m-%d}/{}'.format(base_index, datetime.today(), _type) + self.base_record = dict(infos) + self.base_record['loghub_file'] = loghub_file + self.timeout = timeout + + def mapLogRecord(self, record): + m = self.base_record.copy() + m['timestamp'] = datetime.utcnow().isoformat() + m['level'] = record.levelname + m['msg'] = record.getMessage() + return m + + def emit(self, record): + try: + if sys.version_info[0] < 3: + from httplib import HTTPConnection + else: + from http.client import HTTPConnection + host = self.host + data = json.dumps(self.mapLogRecord(record)) + h = HTTPConnection(host, timeout=self.timeout) + h.putrequest('POST', self.url) + h.putheader('Content-type', 'application/json') + h.putheader("Content-length", str(len(data))) + h.endheaders() + h.send(data.encode('utf-8')) + h.getresponse() + except Exception: + self.handleError(record) diff --git a/dpark/utils/lz4wrapper.py b/dpark/utils/lz4wrapper.py new file mode 100644 index 00000000..5995674e --- /dev/null +++ b/dpark/utils/lz4wrapper.py @@ -0,0 +1,16 @@ +import platform + +if platform.python_implementation() == 'PyPy': + def compress(*a, **kw): + from lz4framed import compress as _compress + return _compress(*a, **kw) + + + def decompress(*a, **kw): + from lz4framed import decompress as _decompress + return _decompress(*a, **kw) + +else: + from lz4framed import compress, decompress + +__all__ = ['compress', 'decompress'] diff --git a/dpark/utils/memory.py b/dpark/utils/memory.py new file mode 100644 index 00000000..1815b00a --- /dev/null +++ b/dpark/utils/memory.py @@ -0,0 +1,126 @@ +import os +import sys +import time +import psutil +import resource +import threading +import platform + +from dpark.utils.log import get_logger + +logger = get_logger(__name__) + +ERROR_TASK_OOM = 3 + + +class MemoryChecker(object): + """ value in MBytes + only used in mesos task + start early + """ + + def __init__(self): + self.rss = 0 + self._stop = False + self.mf = None + self.check = True + self.addation = 0 + self.mem = 100 << 30 + self.ratio = 0.8 + self.thread = None + self.task_id = None + self.oom = False + + @property + def mem_limit_soft(self): + return int(self.mem * self.ratio) + + def add(self, n): + self.addation += n + + @property + def rss_rt(self): + return self.mf().rss + self.addation + + @classmethod + def maxrss(cls): + return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss * 1024 + + def _kill(self, rss, from_main_thread): + template = "task used too much memory: %d MB > %d MB * 1.5," \ + "kill it. use -M argument or taskMemory " \ + "to request more memory." + msg = template % (rss >> 20, self.mem >> 20) + + logger.warning(msg) + if from_main_thread: + os._exit(ERROR_TASK_OOM) + else: + if sys.version[0] == 3: + import _thread + else: + import thread as _thread + self.oom = True + _thread.interrupt_main() + + def after_rotate(self): + limit = self.mem_limit_soft + self.rss = rss = self.rss_rt + if rss > limit * 0.9: + if rss > self.mem * 1.5: + self._kill(rss, from_main_thread=True) + else: + new_limit = max(limit, rss) * 1.1 + self.ratio = new_limit / self.mem * 1.1 + logger.info('after rotate, rss = %d MB, enlarge soft memory limit %d -> %d MB, origin = %d MB', + rss >> 20, + limit >> 20, + self.mem_limit_soft >> 20, + self.mem >> 20) + + def _start(self): + p = psutil.Process() + + logger.debug("start mem check thread") + if hasattr(p, "memory_info"): + self.mf = getattr(p, "memory_info") + else: + self.mf = getattr(p, 'get_memory_info') + mf = self.mf + + def check_mem(): + while not self._stop: + rss = self.rss = (mf().rss + self.addation) # 1ms + if self.check and rss > self.mem * 1.5: + self._kill(rss, from_main_thread=False) + time.sleep(0.1) + + self.thread = t = threading.Thread(target=check_mem) + t.daemon = True + t.start() + + def start(self, task_id, mem_limit_mb): + self._stop = False + self.mem = int(mem_limit_mb) << 20 + self.task_id = task_id + if not self.thread: + self._start() + self.thread.name = "task-%s-checkmem" % (task_id,) + + def stop(self): + self._stop = True + self.thread.join() + self.thread = None + + +def set_oom_score(score=100): + if platform.system() == 'Linux': + pid = os.getpid() + entry = "oom_score_adj" + path = "/proc/{}/{}".format(pid, entry) + + try: + with open(path, "w") as f: + f.write("{}".format(score)) + except: + pass diff --git a/dpark/utils/nested_groupby.py b/dpark/utils/nested_groupby.py new file mode 100644 index 00000000..949d4d70 --- /dev/null +++ b/dpark/utils/nested_groupby.py @@ -0,0 +1,197 @@ +import weakref +import sys +from collections import deque +from dpark.utils.log import get_logger + +logger = get_logger(__name__) + +if sys.version_info[0] < 3: + def next_func(it): + return it.next +else: + def next_func(it): + return it.__next__ + + +def list_value(x): + return x[0], list(x[1]) + + +def list_nested_group(it): + return list(map(list_value, it)) + + +def list_values(x): + return x[0], tuple(map(list, x[1])) + + +def list_nested_cogroup(it): + return list(map(list_values, it)) + + +def group_by_simple(it): + key = None + values = [] + i = -1 + for i, (k, vs) in enumerate(it): + if i == 0: + key = k + values = list(vs) + elif k == key: + values.extend(vs) + else: + yield key, values + key = k + values = list(vs) + if i >= 0: + yield key, values + + +class GroupBySubIter(object): + + def __init__(self, key, next_block_func): + self._key = key + self._next_block_func = next_block_func + self._blocks = None + self._finished = False + + def __iter__(self): + next_block_func = self._next_block_func + key = self._key + while True: + if self._blocks is not None: + if self._blocks: + v = self._blocks.popleft() + else: + break + else: + kv = next_block_func(key) + if kv is None: + break + v = kv[1] + + it = iter(v) + try: + for i in it: + yield i + except StopIteration: + pass + self._finished = True + + def get_all_blocks(self): + if self._finished: + return False + key = self._key + next_block_func = self._next_block_func + blocks = self._blocks = deque() + while True: + kv = next_block_func(key) + if kv is None: + break + blocks.append(kv[1]) + self._finished = True + return len(blocks) + + # def __del__(self): + # print('(Deleting %s)' % self) + + +class GroupByNestedIter(object): + NO_CACHE = False + + def __init__(self, it, owner_info=None): + self._it = iter(it) + self._prev_key = None + self._prev_sub_it = None + self._next_block = None + self.owner_info = owner_info + self.is_cached = False + + def _next_block_for_key(self, k): + """return None when meet a diff key or the end + else return and then update _next_block + """ + if self._next_block is None: # start/end + try: + self._next_block = next(self._it) + except StopIteration: + return + + k_, v_ = self._next_block + if k == k_: + try: + self._next_block = next(self._it) + except StopIteration: + self._next_block = None + pass + return k_, v_ + else: + return + + def _next_key(self, key): + while self._next_block_for_key(key) is not None: + pass + + def __iter__(self): + return self + + def __next__(self): + if self._prev_key is not None: + prev_sub_it = self._prev_sub_it() + if prev_sub_it is not None: + if prev_sub_it.get_all_blocks() and not self.is_cached: + self.is_cached = True + msg = "GroupByNestedIter caching values. owner: %s" % (self.owner_info,) + if GroupByNestedIter.NO_CACHE: # for test + raise Exception(msg) + else: + logger.warning(msg) + + self._next_key(self._prev_key) + + if self._next_block is None: + raise StopIteration + + key = self._next_block[0] + sub_it = GroupBySubIter(key, self._next_block_for_key) + self._prev_key, self._prev_sub_it = key, weakref.ref(sub_it) + return key, iter(sub_it) + + next = __next__ + + +def cogroup_no_dup(iters): + iters = list(map(iter, iters)) + funcs = [next_func(it) for it in iters] + curr = [[[None], i] for i, f in enumerate(funcs)] + + def _key(x): + return x[0][0] + + n = len(iters) + l0 = list([[] for _ in range(n)]) + + r = 0 + + while n: + if r == 0: + min_ = None + else: + min_ = min(curr, key=_key)[0][0] + t = list(l0) + for j, (kv, i) in enumerate(curr): + if kv[0] == min_: + t[i] = kv[1] + yield min_, tuple(t) + + to_del = [] + for j, (kv, i) in enumerate(curr): + if kv[0] == min_: + try: + curr[j] = [funcs[i](), i] + except StopIteration: + to_del.append(j) + n -= 1 + for j, i in enumerate(to_del): + del curr[i - j] + r += 1 diff --git a/dpark/utils/profile.py b/dpark/utils/profile.py new file mode 100644 index 00000000..9d6de53e --- /dev/null +++ b/dpark/utils/profile.py @@ -0,0 +1,43 @@ +from __future__ import absolute_import +from __future__ import print_function +import socket +import getpass +import sys +from cProfile import Profile +from pstats import Stats +from tempfile import NamedTemporaryFile +from functools import wraps +from datetime import datetime + + +def profile(hostname=None, to_stdout=False): + def print_stats(stats): + stats.strip_dirs() + stats.sort_stats('time', 'calls') + stats.print_stats(20) + stats.sort_stats('cumulative') + stats.print_stats(20) + + def decorator(f): + + @wraps(f) + def _(*args, **kwargs): + prof = Profile() + try: + return prof.runcall(f, *args, **kwargs) + finally: + if to_stdout: + stats = Stats(prof) + print_stats(stats) + else: + with NamedTemporaryFile(prefix='dpark_profile_', delete=False) as fd: + print('===\n', datetime.today(), getpass.getuser(), sys.argv[0], file=fd) + stats = Stats(prof, stream=fd) + print_stats(stats) + + return _ + + if hostname is None or socket.gethostname() == hostname: + return decorator + else: + return lambda f: f diff --git a/dpark/utils/recursion.pyx b/dpark/utils/recursion.pyx new file mode 100644 index 00000000..d3fc2ded --- /dev/null +++ b/dpark/utils/recursion.pyx @@ -0,0 +1,11 @@ +from cpython cimport PyInt_FromLong + +cdef extern from "Python.h": + ctypedef struct PyThreadState: + int recursion_depth + PyThreadState *PyThreadState_GET() + +cpdef int get_recursion_depth(): + cdef PyThreadState *tstate = PyThreadState_GET() + return PyInt_FromLong(tstate.recursion_depth - 1) + diff --git a/dpark/utils/tdigest.py b/dpark/utils/tdigest.py new file mode 100644 index 00000000..a0b4b7ee --- /dev/null +++ b/dpark/utils/tdigest.py @@ -0,0 +1,241 @@ +from math import isnan, ceil, pi +from six.moves import range + + +class Centroid(object): + + def __init__(self, x, w=1): + self.__mean = float(x) + self.__count = float(w) + + @property + def mean(self): + return self.__mean + + @property + def count(self): + return self.__count + + def __repr__(self): + return """""" % (self.mean, self.count) + + def __eq__(self, other): + if isinstance(other, Centroid): + return self.mean == other.mean and self.count == other.count + + return False + + +class TDigest(object): + def __init__(self, compression=100, size=None): + self._min = None + self._max = None + self.compression = compression + self._total_weight = 0 + self._weight = [] + self._mean = [] + self._unmerge_weight = 0 + self._tmp_weight = [] + self._tmp_mean = [] + + if size is None: + size = int(2 * ceil(compression)) + 10 + + self._size = size + + @staticmethod + def _weighted_average(x1, w1, x2, w2): + a, b = min(x1, x2), max(x1, x2) + x = float(x1 * w1 + x2 * w2) / (w1 + w2) + return max(a, min(b, x)) + + def __len__(self): + return int(self._total_weight + self._unmerge_weight) + + def __add__(self, other): + if not isinstance(other, TDigest): + raise TypeError('Can not add {} with {}'.format( + self.__class__.__name__, + other.__class__.__name__, + )) + + if len(other) == 0: + return self + + other.compress() + + self._tmp_mean.extend(other._mean) + self._tmp_weight.extend(other._weight) + total = sum(other._weight) + self._unmerge_weight = total + + self.compress() + + return self + + def add(self, x, w=1): + x = float(x) + w = float(w) + if isnan(x): + raise ValueError('Cannot add NaN') + + if len(self._tmp_weight) + len(self._weight) >= self._size - 1: + self.compress() + + self._tmp_weight.append(w) + self._tmp_mean.append(x) + self._unmerge_weight += w + + def compress(self): + if self._unmerge_weight > 0: + self._merge(self._tmp_weight, self._tmp_mean) + self._tmp_weight = [] + self._tmp_mean = [] + self._unmerge_weight = 0 + + def _merge(self, incoming_weight, incoming_mean): + def _argsort(seq): + return sorted(range(len(seq)), key=seq.__getitem__) + + incoming_weight = incoming_weight + self._weight + incoming_mean = incoming_mean + self._mean + + assert incoming_weight + + incoming_order = _argsort(incoming_mean) + + self._total_weight += self._unmerge_weight + + normalizer = self.compression / (pi * self._total_weight) + + mean = [] + weight = [] + mean.append(incoming_mean[incoming_order[0]]) + weight.append(incoming_weight[incoming_order[0]]) + + w_so_far = 0. + for ix in incoming_order[1:]: + proposed_weight = weight[-1] + incoming_weight[ix] + z = proposed_weight * normalizer + q0 = w_so_far / self._total_weight + q2 = (w_so_far + proposed_weight) / self._total_weight + if z * z <= q0 * (1 - q0) and z * z <= q2 * (1 - q2): + weight[-1] += incoming_weight[ix] + mean[-1] = mean[-1] + (incoming_mean[ix] - mean[-1]) * incoming_weight[ix] / weight[-1] + + else: + w_so_far += weight[-1] + mean.append(incoming_mean[ix]) + weight.append(incoming_weight[ix]) + + self._mean = mean + self._weight = weight + # assert sum(weight) == self._total_weight + + if self._total_weight > 0: + self._min = mean[0] if self._min is None else min(self._min, mean[0]) + self._max = mean[-1] if self._max is None else max(self._max, mean[-1]) + + def quantile(self, q): + q = float(q) + if not 0 <= q <= 1: + raise ValueError('q should be in [0, 1], got {}'.format(q)) + + self.compress() + weight = self._weight + mean = self._mean + + if not weight: + return float('nan') + elif len(weight) == 1: + return mean[0] + + index = q * self._total_weight + if index < weight[0] / 2: + return self._min + 2. * index / weight[0] * (mean[0] - self._min) + + weight_so_far = weight[0] / 2. + for i in range(len(weight) - 1): + dw = (weight[i] + weight[i + 1]) / 2. + if weight_so_far + dw > index: + z1 = index - weight_so_far + z2 = weight_so_far + dw - index + return self._weighted_average(mean[i], z2, mean[i + 1], z1) + + weight_so_far += dw + + assert index <= self._total_weight + assert index >= self._total_weight - weight[-1] / 2. + + z1 = index - self._total_weight - weight[-1] / 2. + z2 = weight[-1] / 2. - z1 + return self._weighted_average(mean[-1], z1, self._max, z2) + + def cdf(self, x): + x = float(x) + self.compress() + weight = self._weight + mean = self._mean + + if not weight: + return float('nan') + elif len(weight) == 1: + width = self._max - self._min + if x < self._min: + return 0. + elif x > self._max: + return 1. + elif x - self._min <= width: + return 0.5 + else: + return (x - self._min) / (self._max - self._min) + + if x < self._min: + return 0. + + if x > self._max: + return 1. + + if x <= mean[0]: + if mean[0] - self._min > 0: + return (x - self._min) / (mean[0] - self._min) * weight[0] / self._total_weight / 2. + else: + return 0. + + if x >= mean[-1]: + if self._max - mean[-1] > 0: + return 1. - (self._max - x) / (self._max - mean[-1]) * weight[-1] / self._total_weight / 2. + + else: + return 1. + + weight_so_far = weight[0] / 2. + for it in range(len(weight) - 1): + if mean[it] == x: + w0 = weight_so_far + weight_so_far += sum( + weight[i] + weight[i + 1] + for i in range(it, len(weight) - 1) + if mean[i + 1] == x + ) + return (w0 + weight_so_far) / 2. / self._total_weight + + if mean[it] <= x < mean[it + 1]: + if mean[it + 1] - mean[it] > 0: + dw = (weight[it] + weight[it + 1]) / 2. + return (weight_so_far + + dw * (x - mean[it]) / (mean[it + 1] - mean[it])) / self._total_weight + else: + dw = (weight[it] + weight[it + 1]) / 2. + return weight_so_far + dw / self._total_weight + + weight_so_far += (weight[it] + weight[it + 1]) / 2. + + assert False + + @property + def centroids(self): + self.compress() + weight = self._weight + mean = self._mean + return [Centroid(mean[i], weight[i]) for i in range(len(self._weight))] diff --git a/dpark/web/__init__.py b/dpark/web/__init__.py new file mode 100644 index 00000000..5c5d66c7 --- /dev/null +++ b/dpark/web/__init__.py @@ -0,0 +1,34 @@ +from __future__ import absolute_import +import threading +import socket +from gevent.pywsgi import WSGIServer + +_apps = {} + + +def run(app, port, log=None): + server = WSGIServer(('0.0.0.0', port), app, log=log) + if port not in _apps: + _apps[port] = server + server.serve_forever() + + +def start(app, port=0): + if port == 0: + s = socket.socket() + s.bind(('', 0)) + port = s.getsockname()[1] + s.close() + if port in _apps: + return port + + t = threading.Thread(target=run, args=(app, port)) + t.daemon = True + t.start() + return port + + +def stop(port): + server = _apps.pop(port, None) + if server is not None: + server.stop() diff --git a/dpark/web/ui/__init__.py b/dpark/web/ui/__init__.py new file mode 100644 index 00000000..6b898ba2 --- /dev/null +++ b/dpark/web/ui/__init__.py @@ -0,0 +1,25 @@ +from __future__ import absolute_import +from flask import Flask +from werkzeug.utils import import_string + +blueprints = { + 'index', +} + + +def create_app(context): + app = Flask( + __name__, + static_folder='static', + template_folder='templates' + ) + init_blueprints(app, blueprints) + app.context = context + return app + + +def init_blueprints(app, bps): + for bp in bps: + package = __package__ or __name__ + import_name = '%s.views.%s:bp' % (package, bp) + app.register_blueprint(import_string(import_name)) diff --git a/dpark/web/ui/static/css/bootstrap.min.css b/dpark/web/ui/static/css/bootstrap.min.css new file mode 100755 index 00000000..13cef3d6 --- /dev/null +++ b/dpark/web/ui/static/css/bootstrap.min.css @@ -0,0 +1,874 @@ +/*! + * Bootstrap v2.3.2 + * + * Copyright 2013 Twitter, Inc + * Licensed under the Apache License v2.0 + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Designed and built with all the love in the world @twitter by @mdo and @fat. + */ +.clearfix{*zoom:1;}.clearfix:before,.clearfix:after{display:table;content:"";line-height:0;} +.clearfix:after{clear:both;} +.hide-text{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0;} +.input-block-level{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} +article,aside,details,figcaption,figure,footer,header,hgroup,nav,section{display:block;} +audio,canvas,video{display:inline-block;*display:inline;*zoom:1;} +audio:not([controls]){display:none;} +html{font-size:100%;-webkit-text-size-adjust:100%;-ms-text-size-adjust:100%;} +a:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;} +a:hover,a:active{outline:0;} +sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline;} +sup{top:-0.5em;} +sub{bottom:-0.25em;} +img{max-width:100%;width:auto\9;height:auto;vertical-align:middle;border:0;-ms-interpolation-mode:bicubic;} +#map_canvas img,.google-maps img{max-width:none;} +button,input,select,textarea{margin:0;font-size:100%;vertical-align:middle;} +button,input{*overflow:visible;line-height:normal;} +button::-moz-focus-inner,input::-moz-focus-inner{padding:0;border:0;} +button,html input[type="button"],input[type="reset"],input[type="submit"]{-webkit-appearance:button;cursor:pointer;} +label,select,button,input[type="button"],input[type="reset"],input[type="submit"],input[type="radio"],input[type="checkbox"]{cursor:pointer;} +input[type="search"]{-webkit-box-sizing:content-box;-moz-box-sizing:content-box;box-sizing:content-box;-webkit-appearance:textfield;} +input[type="search"]::-webkit-search-decoration,input[type="search"]::-webkit-search-cancel-button{-webkit-appearance:none;} +textarea{overflow:auto;vertical-align:top;} +@media print{*{text-shadow:none !important;color:#000 !important;background:transparent !important;box-shadow:none !important;} a,a:visited{text-decoration:underline;} a[href]:after{content:" (" attr(href) ")";} abbr[title]:after{content:" (" attr(title) ")";} .ir a:after,a[href^="javascript:"]:after,a[href^="#"]:after{content:"";} pre,blockquote{border:1px solid #999;page-break-inside:avoid;} thead{display:table-header-group;} tr,img{page-break-inside:avoid;} img{max-width:100% !important;} @page {margin:0.5cm;}p,h2,h3{orphans:3;widows:3;} h2,h3{page-break-after:avoid;}}body{margin:0;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:14px;line-height:20px;color:#333333;background-color:#ffffff;} +a{color:#0088cc;text-decoration:none;} +a:hover,a:focus{color:#005580;text-decoration:underline;} +.img-rounded{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;} +.img-polaroid{padding:4px;background-color:#fff;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);-webkit-box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);-moz-box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);box-shadow:0 1px 3px rgba(0, 0, 0, 0.1);} +.img-circle{-webkit-border-radius:500px;-moz-border-radius:500px;border-radius:500px;} +.row{margin-left:-20px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;} +.row:after{clear:both;} +[class*="span"]{float:left;min-height:1px;margin-left:20px;} +.container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px;} +.span12{width:940px;} +.span11{width:860px;} +.span10{width:780px;} +.span9{width:700px;} +.span8{width:620px;} +.span7{width:540px;} +.span6{width:460px;} +.span5{width:380px;} +.span4{width:300px;} +.span3{width:220px;} +.span2{width:140px;} +.span1{width:60px;} +.offset12{margin-left:980px;} +.offset11{margin-left:900px;} +.offset10{margin-left:820px;} +.offset9{margin-left:740px;} +.offset8{margin-left:660px;} +.offset7{margin-left:580px;} +.offset6{margin-left:500px;} +.offset5{margin-left:420px;} +.offset4{margin-left:340px;} +.offset3{margin-left:260px;} +.offset2{margin-left:180px;} +.offset1{margin-left:100px;} +.row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;} +.row-fluid:after{clear:both;} +.row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.127659574468085%;*margin-left:2.074468085106383%;} +.row-fluid [class*="span"]:first-child{margin-left:0;} +.row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.127659574468085%;} +.row-fluid .span12{width:100%;*width:99.94680851063829%;} +.row-fluid .span11{width:91.48936170212765%;*width:91.43617021276594%;} +.row-fluid .span10{width:82.97872340425532%;*width:82.92553191489361%;} +.row-fluid .span9{width:74.46808510638297%;*width:74.41489361702126%;} +.row-fluid .span8{width:65.95744680851064%;*width:65.90425531914893%;} +.row-fluid .span7{width:57.44680851063829%;*width:57.39361702127659%;} +.row-fluid .span6{width:48.93617021276595%;*width:48.88297872340425%;} +.row-fluid .span5{width:40.42553191489362%;*width:40.37234042553192%;} +.row-fluid .span4{width:31.914893617021278%;*width:31.861702127659576%;} +.row-fluid .span3{width:23.404255319148934%;*width:23.351063829787233%;} +.row-fluid .span2{width:14.893617021276595%;*width:14.840425531914894%;} +.row-fluid .span1{width:6.382978723404255%;*width:6.329787234042553%;} +.row-fluid .offset12{margin-left:104.25531914893617%;*margin-left:104.14893617021275%;} +.row-fluid .offset12:first-child{margin-left:102.12765957446808%;*margin-left:102.02127659574467%;} +.row-fluid .offset11{margin-left:95.74468085106382%;*margin-left:95.6382978723404%;} +.row-fluid .offset11:first-child{margin-left:93.61702127659574%;*margin-left:93.51063829787232%;} +.row-fluid .offset10{margin-left:87.23404255319149%;*margin-left:87.12765957446807%;} +.row-fluid .offset10:first-child{margin-left:85.1063829787234%;*margin-left:84.99999999999999%;} +.row-fluid .offset9{margin-left:78.72340425531914%;*margin-left:78.61702127659572%;} +.row-fluid .offset9:first-child{margin-left:76.59574468085106%;*margin-left:76.48936170212764%;} +.row-fluid .offset8{margin-left:70.2127659574468%;*margin-left:70.10638297872339%;} +.row-fluid .offset8:first-child{margin-left:68.08510638297872%;*margin-left:67.9787234042553%;} +.row-fluid .offset7{margin-left:61.70212765957446%;*margin-left:61.59574468085106%;} +.row-fluid .offset7:first-child{margin-left:59.574468085106375%;*margin-left:59.46808510638297%;} +.row-fluid .offset6{margin-left:53.191489361702125%;*margin-left:53.085106382978715%;} +.row-fluid .offset6:first-child{margin-left:51.063829787234035%;*margin-left:50.95744680851063%;} +.row-fluid .offset5{margin-left:44.68085106382979%;*margin-left:44.57446808510638%;} +.row-fluid .offset5:first-child{margin-left:42.5531914893617%;*margin-left:42.4468085106383%;} +.row-fluid .offset4{margin-left:36.170212765957444%;*margin-left:36.06382978723405%;} +.row-fluid .offset4:first-child{margin-left:34.04255319148936%;*margin-left:33.93617021276596%;} +.row-fluid .offset3{margin-left:27.659574468085104%;*margin-left:27.5531914893617%;} +.row-fluid .offset3:first-child{margin-left:25.53191489361702%;*margin-left:25.425531914893618%;} +.row-fluid .offset2{margin-left:19.148936170212764%;*margin-left:19.04255319148936%;} +.row-fluid .offset2:first-child{margin-left:17.02127659574468%;*margin-left:16.914893617021278%;} +.row-fluid .offset1{margin-left:10.638297872340425%;*margin-left:10.53191489361702%;} +.row-fluid .offset1:first-child{margin-left:8.51063829787234%;*margin-left:8.404255319148938%;} +[class*="span"].hide,.row-fluid [class*="span"].hide{display:none;} +[class*="span"].pull-right,.row-fluid [class*="span"].pull-right{float:right;} +.container{margin-right:auto;margin-left:auto;*zoom:1;}.container:before,.container:after{display:table;content:"";line-height:0;} +.container:after{clear:both;} +.container-fluid{padding-right:20px;padding-left:20px;*zoom:1;}.container-fluid:before,.container-fluid:after{display:table;content:"";line-height:0;} +.container-fluid:after{clear:both;} +p{margin:0 0 10px;} +.lead{margin-bottom:20px;font-size:21px;font-weight:200;line-height:30px;} +small{font-size:85%;} +strong{font-weight:bold;} +em{font-style:italic;} +cite{font-style:normal;} +.muted{color:#999999;} +a.muted:hover,a.muted:focus{color:#808080;} +.text-warning{color:#c09853;} +a.text-warning:hover,a.text-warning:focus{color:#a47e3c;} +.text-error{color:#b94a48;} +a.text-error:hover,a.text-error:focus{color:#953b39;} +.text-info{color:#3a87ad;} +a.text-info:hover,a.text-info:focus{color:#2d6987;} +.text-success{color:#468847;} +a.text-success:hover,a.text-success:focus{color:#356635;} +.text-left{text-align:left;} +.text-right{text-align:right;} +.text-center{text-align:center;} +h1,h2,h3,h4,h5,h6{margin:10px 0;font-family:inherit;font-weight:bold;line-height:20px;color:inherit;text-rendering:optimizelegibility;}h1 small,h2 small,h3 small,h4 small,h5 small,h6 small{font-weight:normal;line-height:1;color:#999999;} +h1,h2,h3{line-height:40px;} +h1{font-size:38.5px;} +h2{font-size:31.5px;} +h3{font-size:24.5px;} +h4{font-size:17.5px;} +h5{font-size:14px;} +h6{font-size:11.9px;} +h1 small{font-size:24.5px;} +h2 small{font-size:17.5px;} +h3 small{font-size:14px;} +h4 small{font-size:14px;} +.page-header{padding-bottom:9px;margin:20px 0 30px;border-bottom:1px solid #eeeeee;} +ul,ol{padding:0;margin:0 0 10px 25px;} +ul ul,ul ol,ol ol,ol ul{margin-bottom:0;} +li{line-height:20px;} +ul.unstyled,ol.unstyled{margin-left:0;list-style:none;} +ul.inline,ol.inline{margin-left:0;list-style:none;}ul.inline>li,ol.inline>li{display:inline-block;*display:inline;*zoom:1;padding-left:5px;padding-right:5px;} +dl{margin-bottom:20px;} +dt,dd{line-height:20px;} +dt{font-weight:bold;} +dd{margin-left:10px;} +.dl-horizontal{*zoom:1;}.dl-horizontal:before,.dl-horizontal:after{display:table;content:"";line-height:0;} +.dl-horizontal:after{clear:both;} +.dl-horizontal dt{float:left;width:160px;clear:left;text-align:right;overflow:hidden;text-overflow:ellipsis;white-space:nowrap;} +.dl-horizontal dd{margin-left:180px;} +hr{margin:20px 0;border:0;border-top:1px solid #eeeeee;border-bottom:1px solid #ffffff;} +abbr[title],abbr[data-original-title]{cursor:help;border-bottom:1px dotted #999999;} +abbr.initialism{font-size:90%;text-transform:uppercase;} +blockquote{padding:0 0 0 15px;margin:0 0 20px;border-left:5px solid #eeeeee;}blockquote p{margin-bottom:0;font-size:17.5px;font-weight:300;line-height:1.25;} +blockquote small{display:block;line-height:20px;color:#999999;}blockquote small:before{content:'\2014 \00A0';} +blockquote.pull-right{float:right;padding-right:15px;padding-left:0;border-right:5px solid #eeeeee;border-left:0;}blockquote.pull-right p,blockquote.pull-right small{text-align:right;} +blockquote.pull-right small:before{content:'';} +blockquote.pull-right small:after{content:'\00A0 \2014';} +q:before,q:after,blockquote:before,blockquote:after{content:"";} +address{display:block;margin-bottom:20px;font-style:normal;line-height:20px;} +code,pre{padding:0 3px 2px;font-family:Monaco,Menlo,Consolas,"Courier New",monospace;font-size:12px;color:#333333;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} +code{padding:2px 4px;color:#d14;background-color:#f7f7f9;border:1px solid #e1e1e8;white-space:nowrap;} +pre{display:block;padding:9.5px;margin:0 0 10px;font-size:13px;line-height:20px;word-break:break-all;word-wrap:break-word;white-space:pre;white-space:pre-wrap;background-color:#f5f5f5;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.15);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}pre.prettyprint{margin-bottom:20px;} +pre code{padding:0;color:inherit;white-space:pre;white-space:pre-wrap;background-color:transparent;border:0;} +.pre-scrollable{max-height:340px;overflow-y:scroll;} +.label,.badge{display:inline-block;padding:2px 4px;font-size:11.844px;font-weight:bold;line-height:14px;color:#ffffff;vertical-align:baseline;white-space:nowrap;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#999999;} +.label{-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} +.badge{padding-left:9px;padding-right:9px;-webkit-border-radius:9px;-moz-border-radius:9px;border-radius:9px;} +.label:empty,.badge:empty{display:none;} +a.label:hover,a.label:focus,a.badge:hover,a.badge:focus{color:#ffffff;text-decoration:none;cursor:pointer;} +.label-important,.badge-important{background-color:#b94a48;} +.label-important[href],.badge-important[href]{background-color:#953b39;} +.label-warning,.badge-warning{background-color:#f89406;} +.label-warning[href],.badge-warning[href]{background-color:#c67605;} +.label-success,.badge-success{background-color:#468847;} +.label-success[href],.badge-success[href]{background-color:#356635;} +.label-info,.badge-info{background-color:#3a87ad;} +.label-info[href],.badge-info[href]{background-color:#2d6987;} +.label-inverse,.badge-inverse{background-color:#333333;} +.label-inverse[href],.badge-inverse[href]{background-color:#1a1a1a;} +.btn .label,.btn .badge{position:relative;top:-1px;} +.btn-mini .label,.btn-mini .badge{top:0;} +table{max-width:100%;background-color:transparent;border-collapse:collapse;border-spacing:0;} +.table{width:100%;margin-bottom:20px;}.table th,.table td{padding:8px;line-height:20px;text-align:left;vertical-align:top;border-top:1px solid #dddddd;} +.table th{font-weight:bold;} +.table thead th{vertical-align:bottom;} +.table caption+thead tr:first-child th,.table caption+thead tr:first-child td,.table colgroup+thead tr:first-child th,.table colgroup+thead tr:first-child td,.table thead:first-child tr:first-child th,.table thead:first-child tr:first-child td{border-top:0;} +.table tbody+tbody{border-top:2px solid #dddddd;} +.table .table{background-color:#ffffff;} +.table-condensed th,.table-condensed td{padding:4px 5px;} +.table-bordered{border:1px solid #dddddd;border-collapse:separate;*border-collapse:collapse;border-left:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}.table-bordered th,.table-bordered td{border-left:1px solid #dddddd;} +.table-bordered caption+thead tr:first-child th,.table-bordered caption+tbody tr:first-child th,.table-bordered caption+tbody tr:first-child td,.table-bordered colgroup+thead tr:first-child th,.table-bordered colgroup+tbody tr:first-child th,.table-bordered colgroup+tbody tr:first-child td,.table-bordered thead:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child th,.table-bordered tbody:first-child tr:first-child td{border-top:0;} +.table-bordered thead:first-child tr:first-child>th:first-child,.table-bordered tbody:first-child tr:first-child>td:first-child,.table-bordered tbody:first-child tr:first-child>th:first-child{-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;} +.table-bordered thead:first-child tr:first-child>th:last-child,.table-bordered tbody:first-child tr:first-child>td:last-child,.table-bordered tbody:first-child tr:first-child>th:last-child{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;} +.table-bordered thead:last-child tr:last-child>th:first-child,.table-bordered tbody:last-child tr:last-child>td:first-child,.table-bordered tbody:last-child tr:last-child>th:first-child,.table-bordered tfoot:last-child tr:last-child>td:first-child,.table-bordered tfoot:last-child tr:last-child>th:first-child{-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;} +.table-bordered thead:last-child tr:last-child>th:last-child,.table-bordered tbody:last-child tr:last-child>td:last-child,.table-bordered tbody:last-child tr:last-child>th:last-child,.table-bordered tfoot:last-child tr:last-child>td:last-child,.table-bordered tfoot:last-child tr:last-child>th:last-child{-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;} +.table-bordered tfoot+tbody:last-child tr:last-child td:first-child{-webkit-border-bottom-left-radius:0;-moz-border-radius-bottomleft:0;border-bottom-left-radius:0;} +.table-bordered tfoot+tbody:last-child tr:last-child td:last-child{-webkit-border-bottom-right-radius:0;-moz-border-radius-bottomright:0;border-bottom-right-radius:0;} +.table-bordered caption+thead tr:first-child th:first-child,.table-bordered caption+tbody tr:first-child td:first-child,.table-bordered colgroup+thead tr:first-child th:first-child,.table-bordered colgroup+tbody tr:first-child td:first-child{-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;} +.table-bordered caption+thead tr:first-child th:last-child,.table-bordered caption+tbody tr:first-child td:last-child,.table-bordered colgroup+thead tr:first-child th:last-child,.table-bordered colgroup+tbody tr:first-child td:last-child{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;} +.table-striped tbody>tr:nth-child(odd)>td,.table-striped tbody>tr:nth-child(odd)>th{background-color:#f9f9f9;} +.table-hover tbody tr:hover>td,.table-hover tbody tr:hover>th{background-color:#f5f5f5;} +table td[class*="span"],table th[class*="span"],.row-fluid table td[class*="span"],.row-fluid table th[class*="span"]{display:table-cell;float:none;margin-left:0;} +.table td.span1,.table th.span1{float:none;width:44px;margin-left:0;} +.table td.span2,.table th.span2{float:none;width:124px;margin-left:0;} +.table td.span3,.table th.span3{float:none;width:204px;margin-left:0;} +.table td.span4,.table th.span4{float:none;width:284px;margin-left:0;} +.table td.span5,.table th.span5{float:none;width:364px;margin-left:0;} +.table td.span6,.table th.span6{float:none;width:444px;margin-left:0;} +.table td.span7,.table th.span7{float:none;width:524px;margin-left:0;} +.table td.span8,.table th.span8{float:none;width:604px;margin-left:0;} +.table td.span9,.table th.span9{float:none;width:684px;margin-left:0;} +.table td.span10,.table th.span10{float:none;width:764px;margin-left:0;} +.table td.span11,.table th.span11{float:none;width:844px;margin-left:0;} +.table td.span12,.table th.span12{float:none;width:924px;margin-left:0;} +.table tbody tr.success>td{background-color:#dff0d8;} +.table tbody tr.error>td{background-color:#f2dede;} +.table tbody tr.warning>td{background-color:#fcf8e3;} +.table tbody tr.info>td{background-color:#d9edf7;} +.table-hover tbody tr.success:hover>td{background-color:#d0e9c6;} +.table-hover tbody tr.error:hover>td{background-color:#ebcccc;} +.table-hover tbody tr.warning:hover>td{background-color:#faf2cc;} +.table-hover tbody tr.info:hover>td{background-color:#c4e3f3;} +form{margin:0 0 20px;} +fieldset{padding:0;margin:0;border:0;} +legend{display:block;width:100%;padding:0;margin-bottom:20px;font-size:21px;line-height:40px;color:#333333;border:0;border-bottom:1px solid #e5e5e5;}legend small{font-size:15px;color:#999999;} +label,input,button,select,textarea{font-size:14px;font-weight:normal;line-height:20px;} +input,button,select,textarea{font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;} +label{display:block;margin-bottom:5px;} +select,textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{display:inline-block;height:20px;padding:4px 6px;margin-bottom:10px;font-size:14px;line-height:20px;color:#555555;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;vertical-align:middle;} +input,textarea,.uneditable-input{width:206px;} +textarea{height:auto;} +textarea,input[type="text"],input[type="password"],input[type="datetime"],input[type="datetime-local"],input[type="date"],input[type="month"],input[type="time"],input[type="week"],input[type="number"],input[type="email"],input[type="url"],input[type="search"],input[type="tel"],input[type="color"],.uneditable-input{background-color:#ffffff;border:1px solid #cccccc;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-webkit-transition:border linear .2s, box-shadow linear .2s;-moz-transition:border linear .2s, box-shadow linear .2s;-o-transition:border linear .2s, box-shadow linear .2s;transition:border linear .2s, box-shadow linear .2s;}textarea:focus,input[type="text"]:focus,input[type="password"]:focus,input[type="datetime"]:focus,input[type="datetime-local"]:focus,input[type="date"]:focus,input[type="month"]:focus,input[type="time"]:focus,input[type="week"]:focus,input[type="number"]:focus,input[type="email"]:focus,input[type="url"]:focus,input[type="search"]:focus,input[type="tel"]:focus,input[type="color"]:focus,.uneditable-input:focus{border-color:rgba(82, 168, 236, 0.8);outline:0;outline:thin dotted \9;-webkit-box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);-moz-box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);box-shadow:inset 0 1px 1px rgba(0,0,0,.075), 0 0 8px rgba(82,168,236,.6);} +input[type="radio"],input[type="checkbox"]{margin:4px 0 0;*margin-top:0;margin-top:1px \9;line-height:normal;} +input[type="file"],input[type="image"],input[type="submit"],input[type="reset"],input[type="button"],input[type="radio"],input[type="checkbox"]{width:auto;} +select,input[type="file"]{height:30px;*margin-top:4px;line-height:30px;} +select{width:220px;border:1px solid #cccccc;background-color:#ffffff;} +select[multiple],select[size]{height:auto;} +select:focus,input[type="file"]:focus,input[type="radio"]:focus,input[type="checkbox"]:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;} +.uneditable-input,.uneditable-textarea{color:#999999;background-color:#fcfcfc;border-color:#cccccc;-webkit-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);-moz-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.025);cursor:not-allowed;} +.uneditable-input{overflow:hidden;white-space:nowrap;} +.uneditable-textarea{width:auto;height:auto;} +input:-moz-placeholder,textarea:-moz-placeholder{color:#999999;} +input:-ms-input-placeholder,textarea:-ms-input-placeholder{color:#999999;} +input::-webkit-input-placeholder,textarea::-webkit-input-placeholder{color:#999999;} +.radio,.checkbox{min-height:20px;padding-left:20px;} +.radio input[type="radio"],.checkbox input[type="checkbox"]{float:left;margin-left:-20px;} +.controls>.radio:first-child,.controls>.checkbox:first-child{padding-top:5px;} +.radio.inline,.checkbox.inline{display:inline-block;padding-top:5px;margin-bottom:0;vertical-align:middle;} +.radio.inline+.radio.inline,.checkbox.inline+.checkbox.inline{margin-left:10px;} +.input-mini{width:60px;} +.input-small{width:90px;} +.input-medium{width:150px;} +.input-large{width:210px;} +.input-xlarge{width:270px;} +.input-xxlarge{width:530px;} +input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"]{float:none;margin-left:0;} +.input-append input[class*="span"],.input-append .uneditable-input[class*="span"],.input-prepend input[class*="span"],.input-prepend .uneditable-input[class*="span"],.row-fluid input[class*="span"],.row-fluid select[class*="span"],.row-fluid textarea[class*="span"],.row-fluid .uneditable-input[class*="span"],.row-fluid .input-prepend [class*="span"],.row-fluid .input-append [class*="span"]{display:inline-block;} +input,textarea,.uneditable-input{margin-left:0;} +.controls-row [class*="span"]+[class*="span"]{margin-left:20px;} +input.span12,textarea.span12,.uneditable-input.span12{width:926px;} +input.span11,textarea.span11,.uneditable-input.span11{width:846px;} +input.span10,textarea.span10,.uneditable-input.span10{width:766px;} +input.span9,textarea.span9,.uneditable-input.span9{width:686px;} +input.span8,textarea.span8,.uneditable-input.span8{width:606px;} +input.span7,textarea.span7,.uneditable-input.span7{width:526px;} +input.span6,textarea.span6,.uneditable-input.span6{width:446px;} +input.span5,textarea.span5,.uneditable-input.span5{width:366px;} +input.span4,textarea.span4,.uneditable-input.span4{width:286px;} +input.span3,textarea.span3,.uneditable-input.span3{width:206px;} +input.span2,textarea.span2,.uneditable-input.span2{width:126px;} +input.span1,textarea.span1,.uneditable-input.span1{width:46px;} +.controls-row{*zoom:1;}.controls-row:before,.controls-row:after{display:table;content:"";line-height:0;} +.controls-row:after{clear:both;} +.controls-row [class*="span"],.row-fluid .controls-row [class*="span"]{float:left;} +.controls-row .checkbox[class*="span"],.controls-row .radio[class*="span"]{padding-top:5px;} +input[disabled],select[disabled],textarea[disabled],input[readonly],select[readonly],textarea[readonly]{cursor:not-allowed;background-color:#eeeeee;} +input[type="radio"][disabled],input[type="checkbox"][disabled],input[type="radio"][readonly],input[type="checkbox"][readonly]{background-color:transparent;} +.control-group.warning .control-label,.control-group.warning .help-block,.control-group.warning .help-inline{color:#c09853;} +.control-group.warning .checkbox,.control-group.warning .radio,.control-group.warning input,.control-group.warning select,.control-group.warning textarea{color:#c09853;} +.control-group.warning input,.control-group.warning select,.control-group.warning textarea{border-color:#c09853;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.warning input:focus,.control-group.warning select:focus,.control-group.warning textarea:focus{border-color:#a47e3c;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #dbc59e;} +.control-group.warning .input-prepend .add-on,.control-group.warning .input-append .add-on{color:#c09853;background-color:#fcf8e3;border-color:#c09853;} +.control-group.error .control-label,.control-group.error .help-block,.control-group.error .help-inline{color:#b94a48;} +.control-group.error .checkbox,.control-group.error .radio,.control-group.error input,.control-group.error select,.control-group.error textarea{color:#b94a48;} +.control-group.error input,.control-group.error select,.control-group.error textarea{border-color:#b94a48;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.error input:focus,.control-group.error select:focus,.control-group.error textarea:focus{border-color:#953b39;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #d59392;} +.control-group.error .input-prepend .add-on,.control-group.error .input-append .add-on{color:#b94a48;background-color:#f2dede;border-color:#b94a48;} +.control-group.success .control-label,.control-group.success .help-block,.control-group.success .help-inline{color:#468847;} +.control-group.success .checkbox,.control-group.success .radio,.control-group.success input,.control-group.success select,.control-group.success textarea{color:#468847;} +.control-group.success input,.control-group.success select,.control-group.success textarea{border-color:#468847;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.success input:focus,.control-group.success select:focus,.control-group.success textarea:focus{border-color:#356635;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7aba7b;} +.control-group.success .input-prepend .add-on,.control-group.success .input-append .add-on{color:#468847;background-color:#dff0d8;border-color:#468847;} +.control-group.info .control-label,.control-group.info .help-block,.control-group.info .help-inline{color:#3a87ad;} +.control-group.info .checkbox,.control-group.info .radio,.control-group.info input,.control-group.info select,.control-group.info textarea{color:#3a87ad;} +.control-group.info input,.control-group.info select,.control-group.info textarea{border-color:#3a87ad;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075);}.control-group.info input:focus,.control-group.info select:focus,.control-group.info textarea:focus{border-color:#2d6987;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.075),0 0 6px #7ab5d3;} +.control-group.info .input-prepend .add-on,.control-group.info .input-append .add-on{color:#3a87ad;background-color:#d9edf7;border-color:#3a87ad;} +input:focus:invalid,textarea:focus:invalid,select:focus:invalid{color:#b94a48;border-color:#ee5f5b;}input:focus:invalid:focus,textarea:focus:invalid:focus,select:focus:invalid:focus{border-color:#e9322d;-webkit-box-shadow:0 0 6px #f8b9b7;-moz-box-shadow:0 0 6px #f8b9b7;box-shadow:0 0 6px #f8b9b7;} +.form-actions{padding:19px 20px 20px;margin-top:20px;margin-bottom:20px;background-color:#f5f5f5;border-top:1px solid #e5e5e5;*zoom:1;}.form-actions:before,.form-actions:after{display:table;content:"";line-height:0;} +.form-actions:after{clear:both;} +.help-block,.help-inline{color:#595959;} +.help-block{display:block;margin-bottom:10px;} +.help-inline{display:inline-block;*display:inline;*zoom:1;vertical-align:middle;padding-left:5px;} +.input-append,.input-prepend{display:inline-block;margin-bottom:10px;vertical-align:middle;font-size:0;white-space:nowrap;}.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input,.input-append .dropdown-menu,.input-prepend .dropdown-menu,.input-append .popover,.input-prepend .popover{font-size:14px;} +.input-append input,.input-prepend input,.input-append select,.input-prepend select,.input-append .uneditable-input,.input-prepend .uneditable-input{position:relative;margin-bottom:0;*margin-left:0;vertical-align:top;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;}.input-append input:focus,.input-prepend input:focus,.input-append select:focus,.input-prepend select:focus,.input-append .uneditable-input:focus,.input-prepend .uneditable-input:focus{z-index:2;} +.input-append .add-on,.input-prepend .add-on{display:inline-block;width:auto;height:20px;min-width:16px;padding:4px 5px;font-size:14px;font-weight:normal;line-height:20px;text-align:center;text-shadow:0 1px 0 #ffffff;background-color:#eeeeee;border:1px solid #ccc;} +.input-append .add-on,.input-prepend .add-on,.input-append .btn,.input-prepend .btn,.input-append .btn-group>.dropdown-toggle,.input-prepend .btn-group>.dropdown-toggle{vertical-align:top;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.input-append .active,.input-prepend .active{background-color:#a9dba9;border-color:#46a546;} +.input-prepend .add-on,.input-prepend .btn{margin-right:-1px;} +.input-prepend .add-on:first-child,.input-prepend .btn:first-child{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;} +.input-append input,.input-append select,.input-append .uneditable-input{-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;}.input-append input+.btn-group .btn:last-child,.input-append select+.btn-group .btn:last-child,.input-append .uneditable-input+.btn-group .btn:last-child{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;} +.input-append .add-on,.input-append .btn,.input-append .btn-group{margin-left:-1px;} +.input-append .add-on:last-child,.input-append .btn:last-child,.input-append .btn-group:last-child>.dropdown-toggle{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;} +.input-prepend.input-append input,.input-prepend.input-append select,.input-prepend.input-append .uneditable-input{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;}.input-prepend.input-append input+.btn-group .btn,.input-prepend.input-append select+.btn-group .btn,.input-prepend.input-append .uneditable-input+.btn-group .btn{-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;} +.input-prepend.input-append .add-on:first-child,.input-prepend.input-append .btn:first-child{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;} +.input-prepend.input-append .add-on:last-child,.input-prepend.input-append .btn:last-child{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;} +.input-prepend.input-append .btn-group:first-child{margin-left:0;} +input.search-query{padding-right:14px;padding-right:4px \9;padding-left:14px;padding-left:4px \9;margin-bottom:0;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;} +.form-search .input-append .search-query,.form-search .input-prepend .search-query{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.form-search .input-append .search-query{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px;} +.form-search .input-append .btn{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0;} +.form-search .input-prepend .search-query{-webkit-border-radius:0 14px 14px 0;-moz-border-radius:0 14px 14px 0;border-radius:0 14px 14px 0;} +.form-search .input-prepend .btn{-webkit-border-radius:14px 0 0 14px;-moz-border-radius:14px 0 0 14px;border-radius:14px 0 0 14px;} +.form-search input,.form-inline input,.form-horizontal input,.form-search textarea,.form-inline textarea,.form-horizontal textarea,.form-search select,.form-inline select,.form-horizontal select,.form-search .help-inline,.form-inline .help-inline,.form-horizontal .help-inline,.form-search .uneditable-input,.form-inline .uneditable-input,.form-horizontal .uneditable-input,.form-search .input-prepend,.form-inline .input-prepend,.form-horizontal .input-prepend,.form-search .input-append,.form-inline .input-append,.form-horizontal .input-append{display:inline-block;*display:inline;*zoom:1;margin-bottom:0;vertical-align:middle;} +.form-search .hide,.form-inline .hide,.form-horizontal .hide{display:none;} +.form-search label,.form-inline label,.form-search .btn-group,.form-inline .btn-group{display:inline-block;} +.form-search .input-append,.form-inline .input-append,.form-search .input-prepend,.form-inline .input-prepend{margin-bottom:0;} +.form-search .radio,.form-search .checkbox,.form-inline .radio,.form-inline .checkbox{padding-left:0;margin-bottom:0;vertical-align:middle;} +.form-search .radio input[type="radio"],.form-search .checkbox input[type="checkbox"],.form-inline .radio input[type="radio"],.form-inline .checkbox input[type="checkbox"]{float:left;margin-right:3px;margin-left:0;} +.control-group{margin-bottom:10px;} +legend+.control-group{margin-top:20px;-webkit-margin-top-collapse:separate;} +.form-horizontal .control-group{margin-bottom:20px;*zoom:1;}.form-horizontal .control-group:before,.form-horizontal .control-group:after{display:table;content:"";line-height:0;} +.form-horizontal .control-group:after{clear:both;} +.form-horizontal .control-label{float:left;width:160px;padding-top:5px;text-align:right;} +.form-horizontal .controls{*display:inline-block;*padding-left:20px;margin-left:180px;*margin-left:0;}.form-horizontal .controls:first-child{*padding-left:180px;} +.form-horizontal .help-block{margin-bottom:0;} +.form-horizontal input+.help-block,.form-horizontal select+.help-block,.form-horizontal textarea+.help-block,.form-horizontal .uneditable-input+.help-block,.form-horizontal .input-prepend+.help-block,.form-horizontal .input-append+.help-block{margin-top:10px;} +.form-horizontal .form-actions{padding-left:180px;} +.btn{display:inline-block;*display:inline;*zoom:1;padding:4px 12px;margin-bottom:0;font-size:14px;line-height:20px;text-align:center;vertical-align:middle;cursor:pointer;color:#333333;text-shadow:0 1px 1px rgba(255, 255, 255, 0.75);background-color:#f5f5f5;background-image:-moz-linear-gradient(top, #ffffff, #e6e6e6);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#e6e6e6));background-image:-webkit-linear-gradient(top, #ffffff, #e6e6e6);background-image:-o-linear-gradient(top, #ffffff, #e6e6e6);background-image:linear-gradient(to bottom, #ffffff, #e6e6e6);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#ffe6e6e6', GradientType=0);border-color:#e6e6e6 #e6e6e6 #bfbfbf;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#e6e6e6;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);border:1px solid #cccccc;*border:0;border-bottom-color:#b3b3b3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;*margin-left:.3em;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);}.btn:hover,.btn:focus,.btn:active,.btn.active,.btn.disabled,.btn[disabled]{color:#333333;background-color:#e6e6e6;*background-color:#d9d9d9;} +.btn:active,.btn.active{background-color:#cccccc \9;} +.btn:first-child{*margin-left:0;} +.btn:hover,.btn:focus{color:#333333;text-decoration:none;background-position:0 -15px;-webkit-transition:background-position 0.1s linear;-moz-transition:background-position 0.1s linear;-o-transition:background-position 0.1s linear;transition:background-position 0.1s linear;} +.btn:focus{outline:thin dotted #333;outline:5px auto -webkit-focus-ring-color;outline-offset:-2px;} +.btn.active,.btn:active{background-image:none;outline:0;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);} +.btn.disabled,.btn[disabled]{cursor:default;background-image:none;opacity:0.65;filter:alpha(opacity=65);-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;} +.btn-large{padding:11px 19px;font-size:17.5px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;} +.btn-large [class^="icon-"],.btn-large [class*=" icon-"]{margin-top:4px;} +.btn-small{padding:2px 10px;font-size:11.9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} +.btn-small [class^="icon-"],.btn-small [class*=" icon-"]{margin-top:0;} +.btn-mini [class^="icon-"],.btn-mini [class*=" icon-"]{margin-top:-1px;} +.btn-mini{padding:0 6px;font-size:10.5px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} +.btn-block{display:block;width:100%;padding-left:0;padding-right:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} +.btn-block+.btn-block{margin-top:5px;} +input[type="submit"].btn-block,input[type="reset"].btn-block,input[type="button"].btn-block{width:100%;} +.btn-primary.active,.btn-warning.active,.btn-danger.active,.btn-success.active,.btn-info.active,.btn-inverse.active{color:rgba(255, 255, 255, 0.75);} +.btn-primary{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#006dcc;background-image:-moz-linear-gradient(top, #0088cc, #0044cc);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0044cc));background-image:-webkit-linear-gradient(top, #0088cc, #0044cc);background-image:-o-linear-gradient(top, #0088cc, #0044cc);background-image:linear-gradient(to bottom, #0088cc, #0044cc);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0044cc', GradientType=0);border-color:#0044cc #0044cc #002a80;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#0044cc;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-primary:hover,.btn-primary:focus,.btn-primary:active,.btn-primary.active,.btn-primary.disabled,.btn-primary[disabled]{color:#ffffff;background-color:#0044cc;*background-color:#003bb3;} +.btn-primary:active,.btn-primary.active{background-color:#003399 \9;} +.btn-warning{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#faa732;background-image:-moz-linear-gradient(top, #fbb450, #f89406);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406));background-image:-webkit-linear-gradient(top, #fbb450, #f89406);background-image:-o-linear-gradient(top, #fbb450, #f89406);background-image:linear-gradient(to bottom, #fbb450, #f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0);border-color:#f89406 #f89406 #ad6704;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#f89406;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-warning:hover,.btn-warning:focus,.btn-warning:active,.btn-warning.active,.btn-warning.disabled,.btn-warning[disabled]{color:#ffffff;background-color:#f89406;*background-color:#df8505;} +.btn-warning:active,.btn-warning.active{background-color:#c67605 \9;} +.btn-danger{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#da4f49;background-image:-moz-linear-gradient(top, #ee5f5b, #bd362f);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#bd362f));background-image:-webkit-linear-gradient(top, #ee5f5b, #bd362f);background-image:-o-linear-gradient(top, #ee5f5b, #bd362f);background-image:linear-gradient(to bottom, #ee5f5b, #bd362f);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffbd362f', GradientType=0);border-color:#bd362f #bd362f #802420;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#bd362f;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-danger:hover,.btn-danger:focus,.btn-danger:active,.btn-danger.active,.btn-danger.disabled,.btn-danger[disabled]{color:#ffffff;background-color:#bd362f;*background-color:#a9302a;} +.btn-danger:active,.btn-danger.active{background-color:#942a25 \9;} +.btn-success{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#5bb75b;background-image:-moz-linear-gradient(top, #62c462, #51a351);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#51a351));background-image:-webkit-linear-gradient(top, #62c462, #51a351);background-image:-o-linear-gradient(top, #62c462, #51a351);background-image:linear-gradient(to bottom, #62c462, #51a351);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff51a351', GradientType=0);border-color:#51a351 #51a351 #387038;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#51a351;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-success:hover,.btn-success:focus,.btn-success:active,.btn-success.active,.btn-success.disabled,.btn-success[disabled]{color:#ffffff;background-color:#51a351;*background-color:#499249;} +.btn-success:active,.btn-success.active{background-color:#408140 \9;} +.btn-info{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#49afcd;background-image:-moz-linear-gradient(top, #5bc0de, #2f96b4);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#2f96b4));background-image:-webkit-linear-gradient(top, #5bc0de, #2f96b4);background-image:-o-linear-gradient(top, #5bc0de, #2f96b4);background-image:linear-gradient(to bottom, #5bc0de, #2f96b4);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff2f96b4', GradientType=0);border-color:#2f96b4 #2f96b4 #1f6377;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#2f96b4;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-info:hover,.btn-info:focus,.btn-info:active,.btn-info.active,.btn-info.disabled,.btn-info[disabled]{color:#ffffff;background-color:#2f96b4;*background-color:#2a85a0;} +.btn-info:active,.btn-info.active{background-color:#24748c \9;} +.btn-inverse{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#363636;background-image:-moz-linear-gradient(top, #444444, #222222);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#444444), to(#222222));background-image:-webkit-linear-gradient(top, #444444, #222222);background-image:-o-linear-gradient(top, #444444, #222222);background-image:linear-gradient(to bottom, #444444, #222222);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff444444', endColorstr='#ff222222', GradientType=0);border-color:#222222 #222222 #000000;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#222222;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.btn-inverse:hover,.btn-inverse:focus,.btn-inverse:active,.btn-inverse.active,.btn-inverse.disabled,.btn-inverse[disabled]{color:#ffffff;background-color:#222222;*background-color:#151515;} +.btn-inverse:active,.btn-inverse.active{background-color:#080808 \9;} +button.btn,input[type="submit"].btn{*padding-top:3px;*padding-bottom:3px;}button.btn::-moz-focus-inner,input[type="submit"].btn::-moz-focus-inner{padding:0;border:0;} +button.btn.btn-large,input[type="submit"].btn.btn-large{*padding-top:7px;*padding-bottom:7px;} +button.btn.btn-small,input[type="submit"].btn.btn-small{*padding-top:3px;*padding-bottom:3px;} +button.btn.btn-mini,input[type="submit"].btn.btn-mini{*padding-top:1px;*padding-bottom:1px;} +.btn-link,.btn-link:active,.btn-link[disabled]{background-color:transparent;background-image:none;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;} +.btn-link{border-color:transparent;cursor:pointer;color:#0088cc;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.btn-link:hover,.btn-link:focus{color:#005580;text-decoration:underline;background-color:transparent;} +.btn-link[disabled]:hover,.btn-link[disabled]:focus{color:#333333;text-decoration:none;} +[class^="icon-"],[class*=" icon-"]{display:inline-block;width:14px;height:14px;*margin-right:.3em;line-height:14px;vertical-align:text-top;background-image:url("../img/glyphicons-halflings.png");background-position:14px 14px;background-repeat:no-repeat;margin-top:1px;} +.icon-white,.nav-pills>.active>a>[class^="icon-"],.nav-pills>.active>a>[class*=" icon-"],.nav-list>.active>a>[class^="icon-"],.nav-list>.active>a>[class*=" icon-"],.navbar-inverse .nav>.active>a>[class^="icon-"],.navbar-inverse .nav>.active>a>[class*=" icon-"],.dropdown-menu>li>a:hover>[class^="icon-"],.dropdown-menu>li>a:focus>[class^="icon-"],.dropdown-menu>li>a:hover>[class*=" icon-"],.dropdown-menu>li>a:focus>[class*=" icon-"],.dropdown-menu>.active>a>[class^="icon-"],.dropdown-menu>.active>a>[class*=" icon-"],.dropdown-submenu:hover>a>[class^="icon-"],.dropdown-submenu:focus>a>[class^="icon-"],.dropdown-submenu:hover>a>[class*=" icon-"],.dropdown-submenu:focus>a>[class*=" icon-"]{background-image:url("../img/glyphicons-halflings-white.png");} +.icon-glass{background-position:0 0;} +.icon-music{background-position:-24px 0;} +.icon-search{background-position:-48px 0;} +.icon-envelope{background-position:-72px 0;} +.icon-heart{background-position:-96px 0;} +.icon-star{background-position:-120px 0;} +.icon-star-empty{background-position:-144px 0;} +.icon-user{background-position:-168px 0;} +.icon-film{background-position:-192px 0;} +.icon-th-large{background-position:-216px 0;} +.icon-th{background-position:-240px 0;} +.icon-th-list{background-position:-264px 0;} +.icon-ok{background-position:-288px 0;} +.icon-remove{background-position:-312px 0;} +.icon-zoom-in{background-position:-336px 0;} +.icon-zoom-out{background-position:-360px 0;} +.icon-off{background-position:-384px 0;} +.icon-signal{background-position:-408px 0;} +.icon-cog{background-position:-432px 0;} +.icon-trash{background-position:-456px 0;} +.icon-home{background-position:0 -24px;} +.icon-file{background-position:-24px -24px;} +.icon-time{background-position:-48px -24px;} +.icon-road{background-position:-72px -24px;} +.icon-download-alt{background-position:-96px -24px;} +.icon-download{background-position:-120px -24px;} +.icon-upload{background-position:-144px -24px;} +.icon-inbox{background-position:-168px -24px;} +.icon-play-circle{background-position:-192px -24px;} +.icon-repeat{background-position:-216px -24px;} +.icon-refresh{background-position:-240px -24px;} +.icon-list-alt{background-position:-264px -24px;} +.icon-lock{background-position:-287px -24px;} +.icon-flag{background-position:-312px -24px;} +.icon-headphones{background-position:-336px -24px;} +.icon-volume-off{background-position:-360px -24px;} +.icon-volume-down{background-position:-384px -24px;} +.icon-volume-up{background-position:-408px -24px;} +.icon-qrcode{background-position:-432px -24px;} +.icon-barcode{background-position:-456px -24px;} +.icon-tag{background-position:0 -48px;} +.icon-tags{background-position:-25px -48px;} +.icon-book{background-position:-48px -48px;} +.icon-bookmark{background-position:-72px -48px;} +.icon-print{background-position:-96px -48px;} +.icon-camera{background-position:-120px -48px;} +.icon-font{background-position:-144px -48px;} +.icon-bold{background-position:-167px -48px;} +.icon-italic{background-position:-192px -48px;} +.icon-text-height{background-position:-216px -48px;} +.icon-text-width{background-position:-240px -48px;} +.icon-align-left{background-position:-264px -48px;} +.icon-align-center{background-position:-288px -48px;} +.icon-align-right{background-position:-312px -48px;} +.icon-align-justify{background-position:-336px -48px;} +.icon-list{background-position:-360px -48px;} +.icon-indent-left{background-position:-384px -48px;} +.icon-indent-right{background-position:-408px -48px;} +.icon-facetime-video{background-position:-432px -48px;} +.icon-picture{background-position:-456px -48px;} +.icon-pencil{background-position:0 -72px;} +.icon-map-marker{background-position:-24px -72px;} +.icon-adjust{background-position:-48px -72px;} +.icon-tint{background-position:-72px -72px;} +.icon-edit{background-position:-96px -72px;} +.icon-share{background-position:-120px -72px;} +.icon-check{background-position:-144px -72px;} +.icon-move{background-position:-168px -72px;} +.icon-step-backward{background-position:-192px -72px;} +.icon-fast-backward{background-position:-216px -72px;} +.icon-backward{background-position:-240px -72px;} +.icon-play{background-position:-264px -72px;} +.icon-pause{background-position:-288px -72px;} +.icon-stop{background-position:-312px -72px;} +.icon-forward{background-position:-336px -72px;} +.icon-fast-forward{background-position:-360px -72px;} +.icon-step-forward{background-position:-384px -72px;} +.icon-eject{background-position:-408px -72px;} +.icon-chevron-left{background-position:-432px -72px;} +.icon-chevron-right{background-position:-456px -72px;} +.icon-plus-sign{background-position:0 -96px;} +.icon-minus-sign{background-position:-24px -96px;} +.icon-remove-sign{background-position:-48px -96px;} +.icon-ok-sign{background-position:-72px -96px;} +.icon-question-sign{background-position:-96px -96px;} +.icon-info-sign{background-position:-120px -96px;} +.icon-screenshot{background-position:-144px -96px;} +.icon-remove-circle{background-position:-168px -96px;} +.icon-ok-circle{background-position:-192px -96px;} +.icon-ban-circle{background-position:-216px -96px;} +.icon-arrow-left{background-position:-240px -96px;} +.icon-arrow-right{background-position:-264px -96px;} +.icon-arrow-up{background-position:-289px -96px;} +.icon-arrow-down{background-position:-312px -96px;} +.icon-share-alt{background-position:-336px -96px;} +.icon-resize-full{background-position:-360px -96px;} +.icon-resize-small{background-position:-384px -96px;} +.icon-plus{background-position:-408px -96px;} +.icon-minus{background-position:-433px -96px;} +.icon-asterisk{background-position:-456px -96px;} +.icon-exclamation-sign{background-position:0 -120px;} +.icon-gift{background-position:-24px -120px;} +.icon-leaf{background-position:-48px -120px;} +.icon-fire{background-position:-72px -120px;} +.icon-eye-open{background-position:-96px -120px;} +.icon-eye-close{background-position:-120px -120px;} +.icon-warning-sign{background-position:-144px -120px;} +.icon-plane{background-position:-168px -120px;} +.icon-calendar{background-position:-192px -120px;} +.icon-random{background-position:-216px -120px;width:16px;} +.icon-comment{background-position:-240px -120px;} +.icon-magnet{background-position:-264px -120px;} +.icon-chevron-up{background-position:-288px -120px;} +.icon-chevron-down{background-position:-313px -119px;} +.icon-retweet{background-position:-336px -120px;} +.icon-shopping-cart{background-position:-360px -120px;} +.icon-folder-close{background-position:-384px -120px;width:16px;} +.icon-folder-open{background-position:-408px -120px;width:16px;} +.icon-resize-vertical{background-position:-432px -119px;} +.icon-resize-horizontal{background-position:-456px -118px;} +.icon-hdd{background-position:0 -144px;} +.icon-bullhorn{background-position:-24px -144px;} +.icon-bell{background-position:-48px -144px;} +.icon-certificate{background-position:-72px -144px;} +.icon-thumbs-up{background-position:-96px -144px;} +.icon-thumbs-down{background-position:-120px -144px;} +.icon-hand-right{background-position:-144px -144px;} +.icon-hand-left{background-position:-168px -144px;} +.icon-hand-up{background-position:-192px -144px;} +.icon-hand-down{background-position:-216px -144px;} +.icon-circle-arrow-right{background-position:-240px -144px;} +.icon-circle-arrow-left{background-position:-264px -144px;} +.icon-circle-arrow-up{background-position:-288px -144px;} +.icon-circle-arrow-down{background-position:-312px -144px;} +.icon-globe{background-position:-336px -144px;} +.icon-wrench{background-position:-360px -144px;} +.icon-tasks{background-position:-384px -144px;} +.icon-filter{background-position:-408px -144px;} +.icon-briefcase{background-position:-432px -144px;} +.icon-fullscreen{background-position:-456px -144px;} +.btn-group{position:relative;display:inline-block;*display:inline;*zoom:1;font-size:0;vertical-align:middle;white-space:nowrap;*margin-left:.3em;}.btn-group:first-child{*margin-left:0;} +.btn-group+.btn-group{margin-left:5px;} +.btn-toolbar{font-size:0;margin-top:10px;margin-bottom:10px;}.btn-toolbar>.btn+.btn,.btn-toolbar>.btn-group+.btn,.btn-toolbar>.btn+.btn-group{margin-left:5px;} +.btn-group>.btn{position:relative;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.btn-group>.btn+.btn{margin-left:-1px;} +.btn-group>.btn,.btn-group>.dropdown-menu,.btn-group>.popover{font-size:14px;} +.btn-group>.btn-mini{font-size:10.5px;} +.btn-group>.btn-small{font-size:11.9px;} +.btn-group>.btn-large{font-size:17.5px;} +.btn-group>.btn:first-child{margin-left:0;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;} +.btn-group>.btn:last-child,.btn-group>.dropdown-toggle{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;} +.btn-group>.btn.large:first-child{margin-left:0;-webkit-border-top-left-radius:6px;-moz-border-radius-topleft:6px;border-top-left-radius:6px;-webkit-border-bottom-left-radius:6px;-moz-border-radius-bottomleft:6px;border-bottom-left-radius:6px;} +.btn-group>.btn.large:last-child,.btn-group>.large.dropdown-toggle{-webkit-border-top-right-radius:6px;-moz-border-radius-topright:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;-moz-border-radius-bottomright:6px;border-bottom-right-radius:6px;} +.btn-group>.btn:hover,.btn-group>.btn:focus,.btn-group>.btn:active,.btn-group>.btn.active{z-index:2;} +.btn-group .dropdown-toggle:active,.btn-group.open .dropdown-toggle{outline:0;} +.btn-group>.btn+.dropdown-toggle{padding-left:8px;padding-right:8px;-webkit-box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 1px 0 0 rgba(255,255,255,.125), inset 0 1px 0 rgba(255,255,255,.2), 0 1px 2px rgba(0,0,0,.05);*padding-top:5px;*padding-bottom:5px;} +.btn-group>.btn-mini+.dropdown-toggle{padding-left:5px;padding-right:5px;*padding-top:2px;*padding-bottom:2px;} +.btn-group>.btn-small+.dropdown-toggle{*padding-top:5px;*padding-bottom:4px;} +.btn-group>.btn-large+.dropdown-toggle{padding-left:12px;padding-right:12px;*padding-top:7px;*padding-bottom:7px;} +.btn-group.open .dropdown-toggle{background-image:none;-webkit-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);-moz-box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);box-shadow:inset 0 2px 4px rgba(0,0,0,.15), 0 1px 2px rgba(0,0,0,.05);} +.btn-group.open .btn.dropdown-toggle{background-color:#e6e6e6;} +.btn-group.open .btn-primary.dropdown-toggle{background-color:#0044cc;} +.btn-group.open .btn-warning.dropdown-toggle{background-color:#f89406;} +.btn-group.open .btn-danger.dropdown-toggle{background-color:#bd362f;} +.btn-group.open .btn-success.dropdown-toggle{background-color:#51a351;} +.btn-group.open .btn-info.dropdown-toggle{background-color:#2f96b4;} +.btn-group.open .btn-inverse.dropdown-toggle{background-color:#222222;} +.btn .caret{margin-top:8px;margin-left:0;} +.btn-large .caret{margin-top:6px;} +.btn-large .caret{border-left-width:5px;border-right-width:5px;border-top-width:5px;} +.btn-mini .caret,.btn-small .caret{margin-top:8px;} +.dropup .btn-large .caret{border-bottom-width:5px;} +.btn-primary .caret,.btn-warning .caret,.btn-danger .caret,.btn-info .caret,.btn-success .caret,.btn-inverse .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;} +.btn-group-vertical{display:inline-block;*display:inline;*zoom:1;} +.btn-group-vertical>.btn{display:block;float:none;max-width:100%;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.btn-group-vertical>.btn+.btn{margin-left:0;margin-top:-1px;} +.btn-group-vertical>.btn:first-child{-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0;} +.btn-group-vertical>.btn:last-child{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px;} +.btn-group-vertical>.btn-large:first-child{-webkit-border-radius:6px 6px 0 0;-moz-border-radius:6px 6px 0 0;border-radius:6px 6px 0 0;} +.btn-group-vertical>.btn-large:last-child{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;} +.nav{margin-left:0;margin-bottom:20px;list-style:none;} +.nav>li>a{display:block;} +.nav>li>a:hover,.nav>li>a:focus{text-decoration:none;background-color:#eeeeee;} +.nav>li>a>img{max-width:none;} +.nav>.pull-right{float:right;} +.nav-header{display:block;padding:3px 15px;font-size:11px;font-weight:bold;line-height:20px;color:#999999;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);text-transform:uppercase;} +.nav li+.nav-header{margin-top:9px;} +.nav-list{padding-left:15px;padding-right:15px;margin-bottom:0;} +.nav-list>li>a,.nav-list .nav-header{margin-left:-15px;margin-right:-15px;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);} +.nav-list>li>a{padding:3px 15px;} +.nav-list>.active>a,.nav-list>.active>a:hover,.nav-list>.active>a:focus{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.2);background-color:#0088cc;} +.nav-list [class^="icon-"],.nav-list [class*=" icon-"]{margin-right:2px;} +.nav-list .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #ffffff;} +.nav-tabs,.nav-pills{*zoom:1;}.nav-tabs:before,.nav-pills:before,.nav-tabs:after,.nav-pills:after{display:table;content:"";line-height:0;} +.nav-tabs:after,.nav-pills:after{clear:both;} +.nav-tabs>li,.nav-pills>li{float:left;} +.nav-tabs>li>a,.nav-pills>li>a{padding-right:12px;padding-left:12px;margin-right:2px;line-height:14px;} +.nav-tabs{border-bottom:1px solid #ddd;} +.nav-tabs>li{margin-bottom:-1px;} +.nav-tabs>li>a{padding-top:8px;padding-bottom:8px;line-height:20px;border:1px solid transparent;-webkit-border-radius:4px 4px 0 0;-moz-border-radius:4px 4px 0 0;border-radius:4px 4px 0 0;}.nav-tabs>li>a:hover,.nav-tabs>li>a:focus{border-color:#eeeeee #eeeeee #dddddd;} +.nav-tabs>.active>a,.nav-tabs>.active>a:hover,.nav-tabs>.active>a:focus{color:#555555;background-color:#ffffff;border:1px solid #ddd;border-bottom-color:transparent;cursor:default;} +.nav-pills>li>a{padding-top:8px;padding-bottom:8px;margin-top:2px;margin-bottom:2px;-webkit-border-radius:5px;-moz-border-radius:5px;border-radius:5px;} +.nav-pills>.active>a,.nav-pills>.active>a:hover,.nav-pills>.active>a:focus{color:#ffffff;background-color:#0088cc;} +.nav-stacked>li{float:none;} +.nav-stacked>li>a{margin-right:0;} +.nav-tabs.nav-stacked{border-bottom:0;} +.nav-tabs.nav-stacked>li>a{border:1px solid #ddd;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.nav-tabs.nav-stacked>li:first-child>a{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;} +.nav-tabs.nav-stacked>li:last-child>a{-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;} +.nav-tabs.nav-stacked>li>a:hover,.nav-tabs.nav-stacked>li>a:focus{border-color:#ddd;z-index:2;} +.nav-pills.nav-stacked>li>a{margin-bottom:3px;} +.nav-pills.nav-stacked>li:last-child>a{margin-bottom:1px;} +.nav-tabs .dropdown-menu{-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;} +.nav-pills .dropdown-menu{-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;} +.nav .dropdown-toggle .caret{border-top-color:#0088cc;border-bottom-color:#0088cc;margin-top:6px;} +.nav .dropdown-toggle:hover .caret,.nav .dropdown-toggle:focus .caret{border-top-color:#005580;border-bottom-color:#005580;} +.nav-tabs .dropdown-toggle .caret{margin-top:8px;} +.nav .active .dropdown-toggle .caret{border-top-color:#fff;border-bottom-color:#fff;} +.nav-tabs .active .dropdown-toggle .caret{border-top-color:#555555;border-bottom-color:#555555;} +.nav>.dropdown.active>a:hover,.nav>.dropdown.active>a:focus{cursor:pointer;} +.nav-tabs .open .dropdown-toggle,.nav-pills .open .dropdown-toggle,.nav>li.dropdown.open.active>a:hover,.nav>li.dropdown.open.active>a:focus{color:#ffffff;background-color:#999999;border-color:#999999;} +.nav li.dropdown.open .caret,.nav li.dropdown.open.active .caret,.nav li.dropdown.open a:hover .caret,.nav li.dropdown.open a:focus .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;opacity:1;filter:alpha(opacity=100);} +.tabs-stacked .open>a:hover,.tabs-stacked .open>a:focus{border-color:#999999;} +.tabbable{*zoom:1;}.tabbable:before,.tabbable:after{display:table;content:"";line-height:0;} +.tabbable:after{clear:both;} +.tab-content{overflow:auto;} +.tabs-below>.nav-tabs,.tabs-right>.nav-tabs,.tabs-left>.nav-tabs{border-bottom:0;} +.tab-content>.tab-pane,.pill-content>.pill-pane{display:none;} +.tab-content>.active,.pill-content>.active{display:block;} +.tabs-below>.nav-tabs{border-top:1px solid #ddd;} +.tabs-below>.nav-tabs>li{margin-top:-1px;margin-bottom:0;} +.tabs-below>.nav-tabs>li>a{-webkit-border-radius:0 0 4px 4px;-moz-border-radius:0 0 4px 4px;border-radius:0 0 4px 4px;}.tabs-below>.nav-tabs>li>a:hover,.tabs-below>.nav-tabs>li>a:focus{border-bottom-color:transparent;border-top-color:#ddd;} +.tabs-below>.nav-tabs>.active>a,.tabs-below>.nav-tabs>.active>a:hover,.tabs-below>.nav-tabs>.active>a:focus{border-color:transparent #ddd #ddd #ddd;} +.tabs-left>.nav-tabs>li,.tabs-right>.nav-tabs>li{float:none;} +.tabs-left>.nav-tabs>li>a,.tabs-right>.nav-tabs>li>a{min-width:74px;margin-right:0;margin-bottom:3px;} +.tabs-left>.nav-tabs{float:left;margin-right:19px;border-right:1px solid #ddd;} +.tabs-left>.nav-tabs>li>a{margin-right:-1px;-webkit-border-radius:4px 0 0 4px;-moz-border-radius:4px 0 0 4px;border-radius:4px 0 0 4px;} +.tabs-left>.nav-tabs>li>a:hover,.tabs-left>.nav-tabs>li>a:focus{border-color:#eeeeee #dddddd #eeeeee #eeeeee;} +.tabs-left>.nav-tabs .active>a,.tabs-left>.nav-tabs .active>a:hover,.tabs-left>.nav-tabs .active>a:focus{border-color:#ddd transparent #ddd #ddd;*border-right-color:#ffffff;} +.tabs-right>.nav-tabs{float:right;margin-left:19px;border-left:1px solid #ddd;} +.tabs-right>.nav-tabs>li>a{margin-left:-1px;-webkit-border-radius:0 4px 4px 0;-moz-border-radius:0 4px 4px 0;border-radius:0 4px 4px 0;} +.tabs-right>.nav-tabs>li>a:hover,.tabs-right>.nav-tabs>li>a:focus{border-color:#eeeeee #eeeeee #eeeeee #dddddd;} +.tabs-right>.nav-tabs .active>a,.tabs-right>.nav-tabs .active>a:hover,.tabs-right>.nav-tabs .active>a:focus{border-color:#ddd #ddd #ddd transparent;*border-left-color:#ffffff;} +.nav>.disabled>a{color:#999999;} +.nav>.disabled>a:hover,.nav>.disabled>a:focus{text-decoration:none;background-color:transparent;cursor:default;} +.navbar{overflow:visible;margin-bottom:20px;*position:relative;*z-index:2;} +.navbar-inner{min-height:40px;padding-left:20px;padding-right:20px;background-color:#fafafa;background-image:-moz-linear-gradient(top, #ffffff, #f2f2f2);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ffffff), to(#f2f2f2));background-image:-webkit-linear-gradient(top, #ffffff, #f2f2f2);background-image:-o-linear-gradient(top, #ffffff, #f2f2f2);background-image:linear-gradient(to bottom, #ffffff, #f2f2f2);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffffffff', endColorstr='#fff2f2f2', GradientType=0);border:1px solid #d4d4d4;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);-moz-box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);box-shadow:0 1px 4px rgba(0, 0, 0, 0.065);*zoom:1;}.navbar-inner:before,.navbar-inner:after{display:table;content:"";line-height:0;} +.navbar-inner:after{clear:both;} +.navbar .container{width:auto;} +.nav-collapse.collapse{height:auto;overflow:visible;} +.navbar .brand{float:left;display:block;padding:10px 20px 10px;margin-left:-20px;font-size:20px;font-weight:200;color:#777777;text-shadow:0 1px 0 #ffffff;}.navbar .brand:hover,.navbar .brand:focus{text-decoration:none;} +.navbar-text{margin-bottom:0;line-height:40px;color:#777777;} +.navbar-link{color:#777777;}.navbar-link:hover,.navbar-link:focus{color:#333333;} +.navbar .divider-vertical{height:40px;margin:0 9px;border-left:1px solid #f2f2f2;border-right:1px solid #ffffff;} +.navbar .btn,.navbar .btn-group{margin-top:5px;} +.navbar .btn-group .btn,.navbar .input-prepend .btn,.navbar .input-append .btn,.navbar .input-prepend .btn-group,.navbar .input-append .btn-group{margin-top:0;} +.navbar-form{margin-bottom:0;*zoom:1;}.navbar-form:before,.navbar-form:after{display:table;content:"";line-height:0;} +.navbar-form:after{clear:both;} +.navbar-form input,.navbar-form select,.navbar-form .radio,.navbar-form .checkbox{margin-top:5px;} +.navbar-form input,.navbar-form select,.navbar-form .btn{display:inline-block;margin-bottom:0;} +.navbar-form input[type="image"],.navbar-form input[type="checkbox"],.navbar-form input[type="radio"]{margin-top:3px;} +.navbar-form .input-append,.navbar-form .input-prepend{margin-top:5px;white-space:nowrap;}.navbar-form .input-append input,.navbar-form .input-prepend input{margin-top:0;} +.navbar-search{position:relative;float:left;margin-top:5px;margin-bottom:0;}.navbar-search .search-query{margin-bottom:0;padding:4px 14px;font-family:"Helvetica Neue",Helvetica,Arial,sans-serif;font-size:13px;font-weight:normal;line-height:1;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;} +.navbar-static-top{position:static;margin-bottom:0;}.navbar-static-top .navbar-inner{-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.navbar-fixed-top,.navbar-fixed-bottom{position:fixed;right:0;left:0;z-index:1030;margin-bottom:0;} +.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{border-width:0 0 1px;} +.navbar-fixed-bottom .navbar-inner{border-width:1px 0 0;} +.navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding-left:0;padding-right:0;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;} +.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:940px;} +.navbar-fixed-top{top:0;} +.navbar-fixed-top .navbar-inner,.navbar-static-top .navbar-inner{-webkit-box-shadow:0 1px 10px rgba(0,0,0,.1);-moz-box-shadow:0 1px 10px rgba(0,0,0,.1);box-shadow:0 1px 10px rgba(0,0,0,.1);} +.navbar-fixed-bottom{bottom:0;}.navbar-fixed-bottom .navbar-inner{-webkit-box-shadow:0 -1px 10px rgba(0,0,0,.1);-moz-box-shadow:0 -1px 10px rgba(0,0,0,.1);box-shadow:0 -1px 10px rgba(0,0,0,.1);} +.navbar .nav{position:relative;left:0;display:block;float:left;margin:0 10px 0 0;} +.navbar .nav.pull-right{float:right;margin-right:0;} +.navbar .nav>li{float:left;} +.navbar .nav>li>a{float:none;padding:10px 15px 10px;color:#777777;text-decoration:none;text-shadow:0 1px 0 #ffffff;} +.navbar .nav .dropdown-toggle .caret{margin-top:8px;} +.navbar .nav>li>a:focus,.navbar .nav>li>a:hover{background-color:transparent;color:#333333;text-decoration:none;} +.navbar .nav>.active>a,.navbar .nav>.active>a:hover,.navbar .nav>.active>a:focus{color:#555555;text-decoration:none;background-color:#e5e5e5;-webkit-box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);-moz-box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);box-shadow:inset 0 3px 8px rgba(0, 0, 0, 0.125);} +.navbar .btn-navbar{display:none;float:right;padding:7px 10px;margin-left:5px;margin-right:5px;color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#ededed;background-image:-moz-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#f2f2f2), to(#e5e5e5));background-image:-webkit-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:-o-linear-gradient(top, #f2f2f2, #e5e5e5);background-image:linear-gradient(to bottom, #f2f2f2, #e5e5e5);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff2f2f2', endColorstr='#ffe5e5e5', GradientType=0);border-color:#e5e5e5 #e5e5e5 #bfbfbf;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#e5e5e5;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.075);}.navbar .btn-navbar:hover,.navbar .btn-navbar:focus,.navbar .btn-navbar:active,.navbar .btn-navbar.active,.navbar .btn-navbar.disabled,.navbar .btn-navbar[disabled]{color:#ffffff;background-color:#e5e5e5;*background-color:#d9d9d9;} +.navbar .btn-navbar:active,.navbar .btn-navbar.active{background-color:#cccccc \9;} +.navbar .btn-navbar .icon-bar{display:block;width:18px;height:2px;background-color:#f5f5f5;-webkit-border-radius:1px;-moz-border-radius:1px;border-radius:1px;-webkit-box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);-moz-box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);box-shadow:0 1px 0 rgba(0, 0, 0, 0.25);} +.btn-navbar .icon-bar+.icon-bar{margin-top:3px;} +.navbar .nav>li>.dropdown-menu:before{content:'';display:inline-block;border-left:7px solid transparent;border-right:7px solid transparent;border-bottom:7px solid #ccc;border-bottom-color:rgba(0, 0, 0, 0.2);position:absolute;top:-7px;left:9px;} +.navbar .nav>li>.dropdown-menu:after{content:'';display:inline-block;border-left:6px solid transparent;border-right:6px solid transparent;border-bottom:6px solid #ffffff;position:absolute;top:-6px;left:10px;} +.navbar-fixed-bottom .nav>li>.dropdown-menu:before{border-top:7px solid #ccc;border-top-color:rgba(0, 0, 0, 0.2);border-bottom:0;bottom:-7px;top:auto;} +.navbar-fixed-bottom .nav>li>.dropdown-menu:after{border-top:6px solid #ffffff;border-bottom:0;bottom:-6px;top:auto;} +.navbar .nav li.dropdown>a:hover .caret,.navbar .nav li.dropdown>a:focus .caret{border-top-color:#333333;border-bottom-color:#333333;} +.navbar .nav li.dropdown.open>.dropdown-toggle,.navbar .nav li.dropdown.active>.dropdown-toggle,.navbar .nav li.dropdown.open.active>.dropdown-toggle{background-color:#e5e5e5;color:#555555;} +.navbar .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#777777;border-bottom-color:#777777;} +.navbar .nav li.dropdown.open>.dropdown-toggle .caret,.navbar .nav li.dropdown.active>.dropdown-toggle .caret,.navbar .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#555555;border-bottom-color:#555555;} +.navbar .pull-right>li>.dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right{left:auto;right:0;}.navbar .pull-right>li>.dropdown-menu:before,.navbar .nav>li>.dropdown-menu.pull-right:before{left:auto;right:12px;} +.navbar .pull-right>li>.dropdown-menu:after,.navbar .nav>li>.dropdown-menu.pull-right:after{left:auto;right:13px;} +.navbar .pull-right>li>.dropdown-menu .dropdown-menu,.navbar .nav>li>.dropdown-menu.pull-right .dropdown-menu{left:auto;right:100%;margin-left:0;margin-right:-1px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px;} +.navbar-inverse .navbar-inner{background-color:#1b1b1b;background-image:-moz-linear-gradient(top, #222222, #111111);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#222222), to(#111111));background-image:-webkit-linear-gradient(top, #222222, #111111);background-image:-o-linear-gradient(top, #222222, #111111);background-image:linear-gradient(to bottom, #222222, #111111);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff222222', endColorstr='#ff111111', GradientType=0);border-color:#252525;} +.navbar-inverse .brand,.navbar-inverse .nav>li>a{color:#999999;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);}.navbar-inverse .brand:hover,.navbar-inverse .nav>li>a:hover,.navbar-inverse .brand:focus,.navbar-inverse .nav>li>a:focus{color:#ffffff;} +.navbar-inverse .brand{color:#999999;} +.navbar-inverse .navbar-text{color:#999999;} +.navbar-inverse .nav>li>a:focus,.navbar-inverse .nav>li>a:hover{background-color:transparent;color:#ffffff;} +.navbar-inverse .nav .active>a,.navbar-inverse .nav .active>a:hover,.navbar-inverse .nav .active>a:focus{color:#ffffff;background-color:#111111;} +.navbar-inverse .navbar-link{color:#999999;}.navbar-inverse .navbar-link:hover,.navbar-inverse .navbar-link:focus{color:#ffffff;} +.navbar-inverse .divider-vertical{border-left-color:#111111;border-right-color:#222222;} +.navbar-inverse .nav li.dropdown.open>.dropdown-toggle,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle{background-color:#111111;color:#ffffff;} +.navbar-inverse .nav li.dropdown>a:hover .caret,.navbar-inverse .nav li.dropdown>a:focus .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;} +.navbar-inverse .nav li.dropdown>.dropdown-toggle .caret{border-top-color:#999999;border-bottom-color:#999999;} +.navbar-inverse .nav li.dropdown.open>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.active>.dropdown-toggle .caret,.navbar-inverse .nav li.dropdown.open.active>.dropdown-toggle .caret{border-top-color:#ffffff;border-bottom-color:#ffffff;} +.navbar-inverse .navbar-search .search-query{color:#ffffff;background-color:#515151;border-color:#111111;-webkit-box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);-moz-box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);box-shadow:inset 0 1px 2px rgba(0,0,0,.1), 0 1px 0 rgba(255,255,255,.15);-webkit-transition:none;-moz-transition:none;-o-transition:none;transition:none;}.navbar-inverse .navbar-search .search-query:-moz-placeholder{color:#cccccc;} +.navbar-inverse .navbar-search .search-query:-ms-input-placeholder{color:#cccccc;} +.navbar-inverse .navbar-search .search-query::-webkit-input-placeholder{color:#cccccc;} +.navbar-inverse .navbar-search .search-query:focus,.navbar-inverse .navbar-search .search-query.focused{padding:5px 15px;color:#333333;text-shadow:0 1px 0 #ffffff;background-color:#ffffff;border:0;-webkit-box-shadow:0 0 3px rgba(0, 0, 0, 0.15);-moz-box-shadow:0 0 3px rgba(0, 0, 0, 0.15);box-shadow:0 0 3px rgba(0, 0, 0, 0.15);outline:0;} +.navbar-inverse .btn-navbar{color:#ffffff;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#0e0e0e;background-image:-moz-linear-gradient(top, #151515, #040404);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#151515), to(#040404));background-image:-webkit-linear-gradient(top, #151515, #040404);background-image:-o-linear-gradient(top, #151515, #040404);background-image:linear-gradient(to bottom, #151515, #040404);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff151515', endColorstr='#ff040404', GradientType=0);border-color:#040404 #040404 #000000;border-color:rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.1) rgba(0, 0, 0, 0.25);*background-color:#040404;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);}.navbar-inverse .btn-navbar:hover,.navbar-inverse .btn-navbar:focus,.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active,.navbar-inverse .btn-navbar.disabled,.navbar-inverse .btn-navbar[disabled]{color:#ffffff;background-color:#040404;*background-color:#000000;} +.navbar-inverse .btn-navbar:active,.navbar-inverse .btn-navbar.active{background-color:#000000 \9;} +.breadcrumb{padding:8px 15px;margin:0 0 20px;list-style:none;background-color:#f5f5f5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;}.breadcrumb>li{display:inline-block;*display:inline;*zoom:1;text-shadow:0 1px 0 #ffffff;}.breadcrumb>li>.divider{padding:0 5px;color:#ccc;} +.breadcrumb>.active{color:#999999;} +.pagination{margin:20px 0;} +.pagination ul{display:inline-block;*display:inline;*zoom:1;margin-left:0;margin-bottom:0;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);-moz-box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);box-shadow:0 1px 2px rgba(0, 0, 0, 0.05);} +.pagination ul>li{display:inline;} +.pagination ul>li>a,.pagination ul>li>span{float:left;padding:4px 12px;line-height:20px;text-decoration:none;background-color:#ffffff;border:1px solid #dddddd;border-left-width:0;} +.pagination ul>li>a:hover,.pagination ul>li>a:focus,.pagination ul>.active>a,.pagination ul>.active>span{background-color:#f5f5f5;} +.pagination ul>.active>a,.pagination ul>.active>span{color:#999999;cursor:default;} +.pagination ul>.disabled>span,.pagination ul>.disabled>a,.pagination ul>.disabled>a:hover,.pagination ul>.disabled>a:focus{color:#999999;background-color:transparent;cursor:default;} +.pagination ul>li:first-child>a,.pagination ul>li:first-child>span{border-left-width:1px;-webkit-border-top-left-radius:4px;-moz-border-radius-topleft:4px;border-top-left-radius:4px;-webkit-border-bottom-left-radius:4px;-moz-border-radius-bottomleft:4px;border-bottom-left-radius:4px;} +.pagination ul>li:last-child>a,.pagination ul>li:last-child>span{-webkit-border-top-right-radius:4px;-moz-border-radius-topright:4px;border-top-right-radius:4px;-webkit-border-bottom-right-radius:4px;-moz-border-radius-bottomright:4px;border-bottom-right-radius:4px;} +.pagination-centered{text-align:center;} +.pagination-right{text-align:right;} +.pagination-large ul>li>a,.pagination-large ul>li>span{padding:11px 19px;font-size:17.5px;} +.pagination-large ul>li:first-child>a,.pagination-large ul>li:first-child>span{-webkit-border-top-left-radius:6px;-moz-border-radius-topleft:6px;border-top-left-radius:6px;-webkit-border-bottom-left-radius:6px;-moz-border-radius-bottomleft:6px;border-bottom-left-radius:6px;} +.pagination-large ul>li:last-child>a,.pagination-large ul>li:last-child>span{-webkit-border-top-right-radius:6px;-moz-border-radius-topright:6px;border-top-right-radius:6px;-webkit-border-bottom-right-radius:6px;-moz-border-radius-bottomright:6px;border-bottom-right-radius:6px;} +.pagination-mini ul>li:first-child>a,.pagination-small ul>li:first-child>a,.pagination-mini ul>li:first-child>span,.pagination-small ul>li:first-child>span{-webkit-border-top-left-radius:3px;-moz-border-radius-topleft:3px;border-top-left-radius:3px;-webkit-border-bottom-left-radius:3px;-moz-border-radius-bottomleft:3px;border-bottom-left-radius:3px;} +.pagination-mini ul>li:last-child>a,.pagination-small ul>li:last-child>a,.pagination-mini ul>li:last-child>span,.pagination-small ul>li:last-child>span{-webkit-border-top-right-radius:3px;-moz-border-radius-topright:3px;border-top-right-radius:3px;-webkit-border-bottom-right-radius:3px;-moz-border-radius-bottomright:3px;border-bottom-right-radius:3px;} +.pagination-small ul>li>a,.pagination-small ul>li>span{padding:2px 10px;font-size:11.9px;} +.pagination-mini ul>li>a,.pagination-mini ul>li>span{padding:0 6px;font-size:10.5px;} +.pager{margin:20px 0;list-style:none;text-align:center;*zoom:1;}.pager:before,.pager:after{display:table;content:"";line-height:0;} +.pager:after{clear:both;} +.pager li{display:inline;} +.pager li>a,.pager li>span{display:inline-block;padding:5px 14px;background-color:#fff;border:1px solid #ddd;-webkit-border-radius:15px;-moz-border-radius:15px;border-radius:15px;} +.pager li>a:hover,.pager li>a:focus{text-decoration:none;background-color:#f5f5f5;} +.pager .next>a,.pager .next>span{float:right;} +.pager .previous>a,.pager .previous>span{float:left;} +.pager .disabled>a,.pager .disabled>a:hover,.pager .disabled>a:focus,.pager .disabled>span{color:#999999;background-color:#fff;cursor:default;} +.thumbnails{margin-left:-20px;list-style:none;*zoom:1;}.thumbnails:before,.thumbnails:after{display:table;content:"";line-height:0;} +.thumbnails:after{clear:both;} +.row-fluid .thumbnails{margin-left:0;} +.thumbnails>li{float:left;margin-bottom:20px;margin-left:20px;} +.thumbnail{display:block;padding:4px;line-height:20px;border:1px solid #ddd;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);-moz-box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);box-shadow:0 1px 3px rgba(0, 0, 0, 0.055);-webkit-transition:all 0.2s ease-in-out;-moz-transition:all 0.2s ease-in-out;-o-transition:all 0.2s ease-in-out;transition:all 0.2s ease-in-out;} +a.thumbnail:hover,a.thumbnail:focus{border-color:#0088cc;-webkit-box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);-moz-box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);box-shadow:0 1px 4px rgba(0, 105, 214, 0.25);} +.thumbnail>img{display:block;max-width:100%;margin-left:auto;margin-right:auto;} +.thumbnail .caption{padding:9px;color:#555555;} +.alert{padding:8px 35px 8px 14px;margin-bottom:20px;text-shadow:0 1px 0 rgba(255, 255, 255, 0.5);background-color:#fcf8e3;border:1px solid #fbeed5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} +.alert,.alert h4{color:#c09853;} +.alert h4{margin:0;} +.alert .close{position:relative;top:-2px;right:-21px;line-height:20px;} +.alert-success{background-color:#dff0d8;border-color:#d6e9c6;color:#468847;} +.alert-success h4{color:#468847;} +.alert-danger,.alert-error{background-color:#f2dede;border-color:#eed3d7;color:#b94a48;} +.alert-danger h4,.alert-error h4{color:#b94a48;} +.alert-info{background-color:#d9edf7;border-color:#bce8f1;color:#3a87ad;} +.alert-info h4{color:#3a87ad;} +.alert-block{padding-top:14px;padding-bottom:14px;} +.alert-block>p,.alert-block>ul{margin-bottom:0;} +.alert-block p+p{margin-top:5px;} +@-webkit-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-moz-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-ms-keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}@-o-keyframes progress-bar-stripes{from{background-position:0 0;} to{background-position:40px 0;}}@keyframes progress-bar-stripes{from{background-position:40px 0;} to{background-position:0 0;}}.progress{overflow:hidden;height:20px;margin-bottom:20px;background-color:#f7f7f7;background-image:-moz-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#f5f5f5), to(#f9f9f9));background-image:-webkit-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:-o-linear-gradient(top, #f5f5f5, #f9f9f9);background-image:linear-gradient(to bottom, #f5f5f5, #f9f9f9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fff5f5f5', endColorstr='#fff9f9f9', GradientType=0);-webkit-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);-moz-box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);box-shadow:inset 0 1px 2px rgba(0, 0, 0, 0.1);-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} +.progress .bar{width:0%;height:100%;color:#ffffff;float:left;font-size:12px;text-align:center;text-shadow:0 -1px 0 rgba(0, 0, 0, 0.25);background-color:#0e90d2;background-image:-moz-linear-gradient(top, #149bdf, #0480be);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#149bdf), to(#0480be));background-image:-webkit-linear-gradient(top, #149bdf, #0480be);background-image:-o-linear-gradient(top, #149bdf, #0480be);background-image:linear-gradient(to bottom, #149bdf, #0480be);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff149bdf', endColorstr='#ff0480be', GradientType=0);-webkit-box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);-moz-box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);box-shadow:inset 0 -1px 0 rgba(0, 0, 0, 0.15);-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;-webkit-transition:width 0.6s ease;-moz-transition:width 0.6s ease;-o-transition:width 0.6s ease;transition:width 0.6s ease;} +.progress .bar+.bar{-webkit-box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);-moz-box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);box-shadow:inset 1px 0 0 rgba(0,0,0,.15), inset 0 -1px 0 rgba(0,0,0,.15);} +.progress-striped .bar{background-color:#149bdf;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);-webkit-background-size:40px 40px;-moz-background-size:40px 40px;-o-background-size:40px 40px;background-size:40px 40px;} +.progress.active .bar{-webkit-animation:progress-bar-stripes 2s linear infinite;-moz-animation:progress-bar-stripes 2s linear infinite;-ms-animation:progress-bar-stripes 2s linear infinite;-o-animation:progress-bar-stripes 2s linear infinite;animation:progress-bar-stripes 2s linear infinite;} +.progress-danger .bar,.progress .bar-danger{background-color:#dd514c;background-image:-moz-linear-gradient(top, #ee5f5b, #c43c35);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#ee5f5b), to(#c43c35));background-image:-webkit-linear-gradient(top, #ee5f5b, #c43c35);background-image:-o-linear-gradient(top, #ee5f5b, #c43c35);background-image:linear-gradient(to bottom, #ee5f5b, #c43c35);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ffee5f5b', endColorstr='#ffc43c35', GradientType=0);} +.progress-danger.progress-striped .bar,.progress-striped .bar-danger{background-color:#ee5f5b;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);} +.progress-success .bar,.progress .bar-success{background-color:#5eb95e;background-image:-moz-linear-gradient(top, #62c462, #57a957);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#62c462), to(#57a957));background-image:-webkit-linear-gradient(top, #62c462, #57a957);background-image:-o-linear-gradient(top, #62c462, #57a957);background-image:linear-gradient(to bottom, #62c462, #57a957);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff62c462', endColorstr='#ff57a957', GradientType=0);} +.progress-success.progress-striped .bar,.progress-striped .bar-success{background-color:#62c462;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);} +.progress-info .bar,.progress .bar-info{background-color:#4bb1cf;background-image:-moz-linear-gradient(top, #5bc0de, #339bb9);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#5bc0de), to(#339bb9));background-image:-webkit-linear-gradient(top, #5bc0de, #339bb9);background-image:-o-linear-gradient(top, #5bc0de, #339bb9);background-image:linear-gradient(to bottom, #5bc0de, #339bb9);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff5bc0de', endColorstr='#ff339bb9', GradientType=0);} +.progress-info.progress-striped .bar,.progress-striped .bar-info{background-color:#5bc0de;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);} +.progress-warning .bar,.progress .bar-warning{background-color:#faa732;background-image:-moz-linear-gradient(top, #fbb450, #f89406);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#fbb450), to(#f89406));background-image:-webkit-linear-gradient(top, #fbb450, #f89406);background-image:-o-linear-gradient(top, #fbb450, #f89406);background-image:linear-gradient(to bottom, #fbb450, #f89406);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#fffbb450', endColorstr='#fff89406', GradientType=0);} +.progress-warning.progress-striped .bar,.progress-striped .bar-warning{background-color:#fbb450;background-image:-webkit-gradient(linear, 0 100%, 100% 0, color-stop(0.25, rgba(255, 255, 255, 0.15)), color-stop(0.25, transparent), color-stop(0.5, transparent), color-stop(0.5, rgba(255, 255, 255, 0.15)), color-stop(0.75, rgba(255, 255, 255, 0.15)), color-stop(0.75, transparent), to(transparent));background-image:-webkit-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-moz-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:-o-linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);background-image:linear-gradient(45deg, rgba(255, 255, 255, 0.15) 25%, transparent 25%, transparent 50%, rgba(255, 255, 255, 0.15) 50%, rgba(255, 255, 255, 0.15) 75%, transparent 75%, transparent);} +.hero-unit{padding:60px;margin-bottom:30px;font-size:18px;font-weight:200;line-height:30px;color:inherit;background-color:#eeeeee;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;}.hero-unit h1{margin-bottom:0;font-size:60px;line-height:1;color:inherit;letter-spacing:-1px;} +.hero-unit li{line-height:30px;} +.media,.media-body{overflow:hidden;*overflow:visible;zoom:1;} +.media,.media .media{margin-top:15px;} +.media:first-child{margin-top:0;} +.media-object{display:block;} +.media-heading{margin:0 0 5px;} +.media>.pull-left{margin-right:10px;} +.media>.pull-right{margin-left:10px;} +.media-list{margin-left:0;list-style:none;} +.tooltip{position:absolute;z-index:1030;display:block;visibility:visible;font-size:11px;line-height:1.4;opacity:0;filter:alpha(opacity=0);}.tooltip.in{opacity:0.8;filter:alpha(opacity=80);} +.tooltip.top{margin-top:-3px;padding:5px 0;} +.tooltip.right{margin-left:3px;padding:0 5px;} +.tooltip.bottom{margin-top:3px;padding:5px 0;} +.tooltip.left{margin-left:-3px;padding:0 5px;} +.tooltip-inner{max-width:200px;padding:8px;color:#ffffff;text-align:center;text-decoration:none;background-color:#000000;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} +.tooltip-arrow{position:absolute;width:0;height:0;border-color:transparent;border-style:solid;} +.tooltip.top .tooltip-arrow{bottom:0;left:50%;margin-left:-5px;border-width:5px 5px 0;border-top-color:#000000;} +.tooltip.right .tooltip-arrow{top:50%;left:0;margin-top:-5px;border-width:5px 5px 5px 0;border-right-color:#000000;} +.tooltip.left .tooltip-arrow{top:50%;right:0;margin-top:-5px;border-width:5px 0 5px 5px;border-left-color:#000000;} +.tooltip.bottom .tooltip-arrow{top:0;left:50%;margin-left:-5px;border-width:0 5px 5px;border-bottom-color:#000000;} +.popover{position:absolute;top:0;left:0;z-index:1010;display:none;max-width:276px;padding:1px;text-align:left;background-color:#ffffff;-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-moz-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);white-space:normal;}.popover.top{margin-top:-10px;} +.popover.right{margin-left:10px;} +.popover.bottom{margin-top:10px;} +.popover.left{margin-left:-10px;} +.popover-title{margin:0;padding:8px 14px;font-size:14px;font-weight:normal;line-height:18px;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;-webkit-border-radius:5px 5px 0 0;-moz-border-radius:5px 5px 0 0;border-radius:5px 5px 0 0;}.popover-title:empty{display:none;} +.popover-content{padding:9px 14px;} +.popover .arrow,.popover .arrow:after{position:absolute;display:block;width:0;height:0;border-color:transparent;border-style:solid;} +.popover .arrow{border-width:11px;} +.popover .arrow:after{border-width:10px;content:"";} +.popover.top .arrow{left:50%;margin-left:-11px;border-bottom-width:0;border-top-color:#999;border-top-color:rgba(0, 0, 0, 0.25);bottom:-11px;}.popover.top .arrow:after{bottom:1px;margin-left:-10px;border-bottom-width:0;border-top-color:#ffffff;} +.popover.right .arrow{top:50%;left:-11px;margin-top:-11px;border-left-width:0;border-right-color:#999;border-right-color:rgba(0, 0, 0, 0.25);}.popover.right .arrow:after{left:1px;bottom:-10px;border-left-width:0;border-right-color:#ffffff;} +.popover.bottom .arrow{left:50%;margin-left:-11px;border-top-width:0;border-bottom-color:#999;border-bottom-color:rgba(0, 0, 0, 0.25);top:-11px;}.popover.bottom .arrow:after{top:1px;margin-left:-10px;border-top-width:0;border-bottom-color:#ffffff;} +.popover.left .arrow{top:50%;right:-11px;margin-top:-11px;border-right-width:0;border-left-color:#999;border-left-color:rgba(0, 0, 0, 0.25);}.popover.left .arrow:after{right:1px;border-right-width:0;border-left-color:#ffffff;bottom:-10px;} +.modal-backdrop{position:fixed;top:0;right:0;bottom:0;left:0;z-index:1040;background-color:#000000;}.modal-backdrop.fade{opacity:0;} +.modal-backdrop,.modal-backdrop.fade.in{opacity:0.8;filter:alpha(opacity=80);} +.modal{position:fixed;top:10%;left:50%;z-index:1050;width:560px;margin-left:-280px;background-color:#ffffff;border:1px solid #999;border:1px solid rgba(0, 0, 0, 0.3);*border:1px solid #999;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);-moz-box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);box-shadow:0 3px 7px rgba(0, 0, 0, 0.3);-webkit-background-clip:padding-box;-moz-background-clip:padding-box;background-clip:padding-box;outline:none;}.modal.fade{-webkit-transition:opacity .3s linear, top .3s ease-out;-moz-transition:opacity .3s linear, top .3s ease-out;-o-transition:opacity .3s linear, top .3s ease-out;transition:opacity .3s linear, top .3s ease-out;top:-25%;} +.modal.fade.in{top:10%;} +.modal-header{padding:9px 15px;border-bottom:1px solid #eee;}.modal-header .close{margin-top:2px;} +.modal-header h3{margin:0;line-height:30px;} +.modal-body{position:relative;overflow-y:auto;max-height:400px;padding:15px;} +.modal-form{margin-bottom:0;} +.modal-footer{padding:14px 15px 15px;margin-bottom:0;text-align:right;background-color:#f5f5f5;border-top:1px solid #ddd;-webkit-border-radius:0 0 6px 6px;-moz-border-radius:0 0 6px 6px;border-radius:0 0 6px 6px;-webkit-box-shadow:inset 0 1px 0 #ffffff;-moz-box-shadow:inset 0 1px 0 #ffffff;box-shadow:inset 0 1px 0 #ffffff;*zoom:1;}.modal-footer:before,.modal-footer:after{display:table;content:"";line-height:0;} +.modal-footer:after{clear:both;} +.modal-footer .btn+.btn{margin-left:5px;margin-bottom:0;} +.modal-footer .btn-group .btn+.btn{margin-left:-1px;} +.modal-footer .btn-block+.btn-block{margin-left:0;} +.dropup,.dropdown{position:relative;} +.dropdown-toggle{*margin-bottom:-3px;} +.dropdown-toggle:active,.open .dropdown-toggle{outline:0;} +.caret{display:inline-block;width:0;height:0;vertical-align:top;border-top:4px solid #000000;border-right:4px solid transparent;border-left:4px solid transparent;content:"";} +.dropdown .caret{margin-top:8px;margin-left:2px;} +.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:160px;padding:5px 0;margin:2px 0 0;list-style:none;background-color:#ffffff;border:1px solid #ccc;border:1px solid rgba(0, 0, 0, 0.2);*border-right-width:2px;*border-bottom-width:2px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;-webkit-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-moz-box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);box-shadow:0 5px 10px rgba(0, 0, 0, 0.2);-webkit-background-clip:padding-box;-moz-background-clip:padding;background-clip:padding-box;}.dropdown-menu.pull-right{right:0;left:auto;} +.dropdown-menu .divider{*width:100%;height:1px;margin:9px 1px;*margin:-5px 0 5px;overflow:hidden;background-color:#e5e5e5;border-bottom:1px solid #ffffff;} +.dropdown-menu>li>a{display:block;padding:3px 20px;clear:both;font-weight:normal;line-height:20px;color:#333333;white-space:nowrap;} +.dropdown-menu>li>a:hover,.dropdown-menu>li>a:focus,.dropdown-submenu:hover>a,.dropdown-submenu:focus>a{text-decoration:none;color:#ffffff;background-color:#0081c2;background-image:-moz-linear-gradient(top, #0088cc, #0077b3);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));background-image:-webkit-linear-gradient(top, #0088cc, #0077b3);background-image:-o-linear-gradient(top, #0088cc, #0077b3);background-image:linear-gradient(to bottom, #0088cc, #0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);} +.dropdown-menu>.active>a,.dropdown-menu>.active>a:hover,.dropdown-menu>.active>a:focus{color:#ffffff;text-decoration:none;outline:0;background-color:#0081c2;background-image:-moz-linear-gradient(top, #0088cc, #0077b3);background-image:-webkit-gradient(linear, 0 0, 0 100%, from(#0088cc), to(#0077b3));background-image:-webkit-linear-gradient(top, #0088cc, #0077b3);background-image:-o-linear-gradient(top, #0088cc, #0077b3);background-image:linear-gradient(to bottom, #0088cc, #0077b3);background-repeat:repeat-x;filter:progid:DXImageTransform.Microsoft.gradient(startColorstr='#ff0088cc', endColorstr='#ff0077b3', GradientType=0);} +.dropdown-menu>.disabled>a,.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{color:#999999;} +.dropdown-menu>.disabled>a:hover,.dropdown-menu>.disabled>a:focus{text-decoration:none;background-color:transparent;background-image:none;filter:progid:DXImageTransform.Microsoft.gradient(enabled = false);cursor:default;} +.open{*z-index:1000;}.open>.dropdown-menu{display:block;} +.dropdown-backdrop{position:fixed;left:0;right:0;bottom:0;top:0;z-index:990;} +.pull-right>.dropdown-menu{right:0;left:auto;} +.dropup .caret,.navbar-fixed-bottom .dropdown .caret{border-top:0;border-bottom:4px solid #000000;content:"";} +.dropup .dropdown-menu,.navbar-fixed-bottom .dropdown .dropdown-menu{top:auto;bottom:100%;margin-bottom:1px;} +.dropdown-submenu{position:relative;} +.dropdown-submenu>.dropdown-menu{top:0;left:100%;margin-top:-6px;margin-left:-1px;-webkit-border-radius:0 6px 6px 6px;-moz-border-radius:0 6px 6px 6px;border-radius:0 6px 6px 6px;} +.dropdown-submenu:hover>.dropdown-menu{display:block;} +.dropup .dropdown-submenu>.dropdown-menu{top:auto;bottom:0;margin-top:0;margin-bottom:-2px;-webkit-border-radius:5px 5px 5px 0;-moz-border-radius:5px 5px 5px 0;border-radius:5px 5px 5px 0;} +.dropdown-submenu>a:after{display:block;content:" ";float:right;width:0;height:0;border-color:transparent;border-style:solid;border-width:5px 0 5px 5px;border-left-color:#cccccc;margin-top:5px;margin-right:-10px;} +.dropdown-submenu:hover>a:after{border-left-color:#ffffff;} +.dropdown-submenu.pull-left{float:none;}.dropdown-submenu.pull-left>.dropdown-menu{left:-100%;margin-left:10px;-webkit-border-radius:6px 0 6px 6px;-moz-border-radius:6px 0 6px 6px;border-radius:6px 0 6px 6px;} +.dropdown .dropdown-menu .nav-header{padding-left:20px;padding-right:20px;} +.typeahead{z-index:1051;margin-top:2px;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} +.accordion{margin-bottom:20px;} +.accordion-group{margin-bottom:2px;border:1px solid #e5e5e5;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} +.accordion-heading{border-bottom:0;} +.accordion-heading .accordion-toggle{display:block;padding:8px 15px;} +.accordion-toggle{cursor:pointer;} +.accordion-inner{padding:9px 15px;border-top:1px solid #e5e5e5;} +.carousel{position:relative;margin-bottom:20px;line-height:1;} +.carousel-inner{overflow:hidden;width:100%;position:relative;} +.carousel-inner>.item{display:none;position:relative;-webkit-transition:0.6s ease-in-out left;-moz-transition:0.6s ease-in-out left;-o-transition:0.6s ease-in-out left;transition:0.6s ease-in-out left;}.carousel-inner>.item>img,.carousel-inner>.item>a>img{display:block;line-height:1;} +.carousel-inner>.active,.carousel-inner>.next,.carousel-inner>.prev{display:block;} +.carousel-inner>.active{left:0;} +.carousel-inner>.next,.carousel-inner>.prev{position:absolute;top:0;width:100%;} +.carousel-inner>.next{left:100%;} +.carousel-inner>.prev{left:-100%;} +.carousel-inner>.next.left,.carousel-inner>.prev.right{left:0;} +.carousel-inner>.active.left{left:-100%;} +.carousel-inner>.active.right{left:100%;} +.carousel-control{position:absolute;top:40%;left:15px;width:40px;height:40px;margin-top:-20px;font-size:60px;font-weight:100;line-height:30px;color:#ffffff;text-align:center;background:#222222;border:3px solid #ffffff;-webkit-border-radius:23px;-moz-border-radius:23px;border-radius:23px;opacity:0.5;filter:alpha(opacity=50);}.carousel-control.right{left:auto;right:15px;} +.carousel-control:hover,.carousel-control:focus{color:#ffffff;text-decoration:none;opacity:0.9;filter:alpha(opacity=90);} +.carousel-indicators{position:absolute;top:15px;right:15px;z-index:5;margin:0;list-style:none;}.carousel-indicators li{display:block;float:left;width:10px;height:10px;margin-left:5px;text-indent:-999px;background-color:#ccc;background-color:rgba(255, 255, 255, 0.25);border-radius:5px;} +.carousel-indicators .active{background-color:#fff;} +.carousel-caption{position:absolute;left:0;right:0;bottom:0;padding:15px;background:#333333;background:rgba(0, 0, 0, 0.75);} +.carousel-caption h4,.carousel-caption p{color:#ffffff;line-height:20px;} +.carousel-caption h4{margin:0 0 5px;} +.carousel-caption p{margin-bottom:0;} +.well{min-height:20px;padding:19px;margin-bottom:20px;background-color:#f5f5f5;border:1px solid #e3e3e3;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;-webkit-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);-moz-box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);box-shadow:inset 0 1px 1px rgba(0, 0, 0, 0.05);}.well blockquote{border-color:#ddd;border-color:rgba(0, 0, 0, 0.15);} +.well-large{padding:24px;-webkit-border-radius:6px;-moz-border-radius:6px;border-radius:6px;} +.well-small{padding:9px;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} +.close{float:right;font-size:20px;font-weight:bold;line-height:20px;color:#000000;text-shadow:0 1px 0 #ffffff;opacity:0.2;filter:alpha(opacity=20);}.close:hover,.close:focus{color:#000000;text-decoration:none;cursor:pointer;opacity:0.4;filter:alpha(opacity=40);} +button.close{padding:0;cursor:pointer;background:transparent;border:0;-webkit-appearance:none;} +.pull-right{float:right;} +.pull-left{float:left;} +.hide{display:none;} +.show{display:block;} +.invisible{visibility:hidden;} +.affix{position:fixed;} +.fade{opacity:0;-webkit-transition:opacity 0.15s linear;-moz-transition:opacity 0.15s linear;-o-transition:opacity 0.15s linear;transition:opacity 0.15s linear;}.fade.in{opacity:1;} +.collapse{position:relative;height:0;overflow:hidden;-webkit-transition:height 0.35s ease;-moz-transition:height 0.35s ease;-o-transition:height 0.35s ease;transition:height 0.35s ease;}.collapse.in{height:auto;} +@-ms-viewport{width:device-width;}.hidden{display:none;visibility:hidden;} +.visible-phone{display:none !important;} +.visible-tablet{display:none !important;} +.hidden-desktop{display:none !important;} +.visible-desktop{display:inherit !important;} +@media (min-width:768px) and (max-width:979px){.hidden-desktop{display:inherit !important;} .visible-desktop{display:none !important ;} .visible-tablet{display:inherit !important;} .hidden-tablet{display:none !important;}}@media (max-width:767px){.hidden-desktop{display:inherit !important;} .visible-desktop{display:none !important;} .visible-phone{display:inherit !important;} .hidden-phone{display:none !important;}}.visible-print{display:none !important;} +@media print{.visible-print{display:inherit !important;} .hidden-print{display:none !important;}}@media (max-width:767px){body{padding-left:20px;padding-right:20px;} .navbar-fixed-top,.navbar-fixed-bottom,.navbar-static-top{margin-left:-20px;margin-right:-20px;} .container-fluid{padding:0;} .dl-horizontal dt{float:none;clear:none;width:auto;text-align:left;} .dl-horizontal dd{margin-left:0;} .container{width:auto;} .row-fluid{width:100%;} .row,.thumbnails{margin-left:0;} .thumbnails>li{float:none;margin-left:0;} [class*="span"],.uneditable-input[class*="span"],.row-fluid [class*="span"]{float:none;display:block;width:100%;margin-left:0;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .span12,.row-fluid .span12{width:100%;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .row-fluid [class*="offset"]:first-child{margin-left:0;} .input-large,.input-xlarge,.input-xxlarge,input[class*="span"],select[class*="span"],textarea[class*="span"],.uneditable-input{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;} .input-prepend input,.input-append input,.input-prepend input[class*="span"],.input-append input[class*="span"]{display:inline-block;width:auto;} .controls-row [class*="span"]+[class*="span"]{margin-left:0;} .modal{position:fixed;top:20px;left:20px;right:20px;width:auto;margin:0;}.modal.fade{top:-100px;} .modal.fade.in{top:20px;}}@media (max-width:480px){.nav-collapse{-webkit-transform:translate3d(0, 0, 0);} .page-header h1 small{display:block;line-height:20px;} input[type="checkbox"],input[type="radio"]{border:1px solid #ccc;} .form-horizontal .control-label{float:none;width:auto;padding-top:0;text-align:left;} .form-horizontal .controls{margin-left:0;} .form-horizontal .control-list{padding-top:0;} .form-horizontal .form-actions{padding-left:10px;padding-right:10px;} .media .pull-left,.media .pull-right{float:none;display:block;margin-bottom:10px;} .media-object{margin-right:0;margin-left:0;} .modal{top:10px;left:10px;right:10px;} .modal-header .close{padding:10px;margin:-10px;} .carousel-caption{position:static;}}@media (min-width:768px) and (max-width:979px){.row{margin-left:-20px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;} .row:after{clear:both;} [class*="span"]{float:left;min-height:1px;margin-left:20px;} .container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:724px;} .span12{width:724px;} .span11{width:662px;} .span10{width:600px;} .span9{width:538px;} .span8{width:476px;} .span7{width:414px;} .span6{width:352px;} .span5{width:290px;} .span4{width:228px;} .span3{width:166px;} .span2{width:104px;} .span1{width:42px;} .offset12{margin-left:764px;} .offset11{margin-left:702px;} .offset10{margin-left:640px;} .offset9{margin-left:578px;} .offset8{margin-left:516px;} .offset7{margin-left:454px;} .offset6{margin-left:392px;} .offset5{margin-left:330px;} .offset4{margin-left:268px;} .offset3{margin-left:206px;} .offset2{margin-left:144px;} .offset1{margin-left:82px;} .row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;} .row-fluid:after{clear:both;} .row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.7624309392265194%;*margin-left:2.709239449864817%;} .row-fluid [class*="span"]:first-child{margin-left:0;} .row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.7624309392265194%;} .row-fluid .span12{width:100%;*width:99.94680851063829%;} .row-fluid .span11{width:91.43646408839778%;*width:91.38327259903608%;} .row-fluid .span10{width:82.87292817679558%;*width:82.81973668743387%;} .row-fluid .span9{width:74.30939226519337%;*width:74.25620077583166%;} .row-fluid .span8{width:65.74585635359117%;*width:65.69266486422946%;} .row-fluid .span7{width:57.18232044198895%;*width:57.12912895262725%;} .row-fluid .span6{width:48.61878453038674%;*width:48.56559304102504%;} .row-fluid .span5{width:40.05524861878453%;*width:40.00205712942283%;} .row-fluid .span4{width:31.491712707182323%;*width:31.43852121782062%;} .row-fluid .span3{width:22.92817679558011%;*width:22.87498530621841%;} .row-fluid .span2{width:14.3646408839779%;*width:14.311449394616199%;} .row-fluid .span1{width:5.801104972375691%;*width:5.747913483013988%;} .row-fluid .offset12{margin-left:105.52486187845304%;*margin-left:105.41847889972962%;} .row-fluid .offset12:first-child{margin-left:102.76243093922652%;*margin-left:102.6560479605031%;} .row-fluid .offset11{margin-left:96.96132596685082%;*margin-left:96.8549429881274%;} .row-fluid .offset11:first-child{margin-left:94.1988950276243%;*margin-left:94.09251204890089%;} .row-fluid .offset10{margin-left:88.39779005524862%;*margin-left:88.2914070765252%;} .row-fluid .offset10:first-child{margin-left:85.6353591160221%;*margin-left:85.52897613729868%;} .row-fluid .offset9{margin-left:79.8342541436464%;*margin-left:79.72787116492299%;} .row-fluid .offset9:first-child{margin-left:77.07182320441989%;*margin-left:76.96544022569647%;} .row-fluid .offset8{margin-left:71.2707182320442%;*margin-left:71.16433525332079%;} .row-fluid .offset8:first-child{margin-left:68.50828729281768%;*margin-left:68.40190431409427%;} .row-fluid .offset7{margin-left:62.70718232044199%;*margin-left:62.600799341718584%;} .row-fluid .offset7:first-child{margin-left:59.94475138121547%;*margin-left:59.838368402492065%;} .row-fluid .offset6{margin-left:54.14364640883978%;*margin-left:54.037263430116376%;} .row-fluid .offset6:first-child{margin-left:51.38121546961326%;*margin-left:51.27483249088986%;} .row-fluid .offset5{margin-left:45.58011049723757%;*margin-left:45.47372751851417%;} .row-fluid .offset5:first-child{margin-left:42.81767955801105%;*margin-left:42.71129657928765%;} .row-fluid .offset4{margin-left:37.01657458563536%;*margin-left:36.91019160691196%;} .row-fluid .offset4:first-child{margin-left:34.25414364640884%;*margin-left:34.14776066768544%;} .row-fluid .offset3{margin-left:28.45303867403315%;*margin-left:28.346655695309746%;} .row-fluid .offset3:first-child{margin-left:25.69060773480663%;*margin-left:25.584224756083227%;} .row-fluid .offset2{margin-left:19.88950276243094%;*margin-left:19.783119783707537%;} .row-fluid .offset2:first-child{margin-left:17.12707182320442%;*margin-left:17.02068884448102%;} .row-fluid .offset1{margin-left:11.32596685082873%;*margin-left:11.219583872105325%;} .row-fluid .offset1:first-child{margin-left:8.56353591160221%;*margin-left:8.457152932878806%;} input,textarea,.uneditable-input{margin-left:0;} .controls-row [class*="span"]+[class*="span"]{margin-left:20px;} input.span12,textarea.span12,.uneditable-input.span12{width:710px;} input.span11,textarea.span11,.uneditable-input.span11{width:648px;} input.span10,textarea.span10,.uneditable-input.span10{width:586px;} input.span9,textarea.span9,.uneditable-input.span9{width:524px;} input.span8,textarea.span8,.uneditable-input.span8{width:462px;} input.span7,textarea.span7,.uneditable-input.span7{width:400px;} input.span6,textarea.span6,.uneditable-input.span6{width:338px;} input.span5,textarea.span5,.uneditable-input.span5{width:276px;} input.span4,textarea.span4,.uneditable-input.span4{width:214px;} input.span3,textarea.span3,.uneditable-input.span3{width:152px;} input.span2,textarea.span2,.uneditable-input.span2{width:90px;} input.span1,textarea.span1,.uneditable-input.span1{width:28px;}}@media (min-width:1200px){.row{margin-left:-30px;*zoom:1;}.row:before,.row:after{display:table;content:"";line-height:0;} .row:after{clear:both;} [class*="span"]{float:left;min-height:1px;margin-left:30px;} .container,.navbar-static-top .container,.navbar-fixed-top .container,.navbar-fixed-bottom .container{width:1170px;} .span12{width:1170px;} .span11{width:1070px;} .span10{width:970px;} .span9{width:870px;} .span8{width:770px;} .span7{width:670px;} .span6{width:570px;} .span5{width:470px;} .span4{width:370px;} .span3{width:270px;} .span2{width:170px;} .span1{width:70px;} .offset12{margin-left:1230px;} .offset11{margin-left:1130px;} .offset10{margin-left:1030px;} .offset9{margin-left:930px;} .offset8{margin-left:830px;} .offset7{margin-left:730px;} .offset6{margin-left:630px;} .offset5{margin-left:530px;} .offset4{margin-left:430px;} .offset3{margin-left:330px;} .offset2{margin-left:230px;} .offset1{margin-left:130px;} .row-fluid{width:100%;*zoom:1;}.row-fluid:before,.row-fluid:after{display:table;content:"";line-height:0;} .row-fluid:after{clear:both;} .row-fluid [class*="span"]{display:block;width:100%;min-height:30px;-webkit-box-sizing:border-box;-moz-box-sizing:border-box;box-sizing:border-box;float:left;margin-left:2.564102564102564%;*margin-left:2.5109110747408616%;} .row-fluid [class*="span"]:first-child{margin-left:0;} .row-fluid .controls-row [class*="span"]+[class*="span"]{margin-left:2.564102564102564%;} .row-fluid .span12{width:100%;*width:99.94680851063829%;} .row-fluid .span11{width:91.45299145299145%;*width:91.39979996362975%;} .row-fluid .span10{width:82.90598290598291%;*width:82.8527914166212%;} .row-fluid .span9{width:74.35897435897436%;*width:74.30578286961266%;} .row-fluid .span8{width:65.81196581196582%;*width:65.75877432260411%;} .row-fluid .span7{width:57.26495726495726%;*width:57.21176577559556%;} .row-fluid .span6{width:48.717948717948715%;*width:48.664757228587014%;} .row-fluid .span5{width:40.17094017094017%;*width:40.11774868157847%;} .row-fluid .span4{width:31.623931623931625%;*width:31.570740134569924%;} .row-fluid .span3{width:23.076923076923077%;*width:23.023731587561375%;} .row-fluid .span2{width:14.52991452991453%;*width:14.476723040552828%;} .row-fluid .span1{width:5.982905982905983%;*width:5.929714493544281%;} .row-fluid .offset12{margin-left:105.12820512820512%;*margin-left:105.02182214948171%;} .row-fluid .offset12:first-child{margin-left:102.56410256410257%;*margin-left:102.45771958537915%;} .row-fluid .offset11{margin-left:96.58119658119658%;*margin-left:96.47481360247316%;} .row-fluid .offset11:first-child{margin-left:94.01709401709402%;*margin-left:93.91071103837061%;} .row-fluid .offset10{margin-left:88.03418803418803%;*margin-left:87.92780505546462%;} .row-fluid .offset10:first-child{margin-left:85.47008547008548%;*margin-left:85.36370249136206%;} .row-fluid .offset9{margin-left:79.48717948717949%;*margin-left:79.38079650845607%;} .row-fluid .offset9:first-child{margin-left:76.92307692307693%;*margin-left:76.81669394435352%;} .row-fluid .offset8{margin-left:70.94017094017094%;*margin-left:70.83378796144753%;} .row-fluid .offset8:first-child{margin-left:68.37606837606839%;*margin-left:68.26968539734497%;} .row-fluid .offset7{margin-left:62.393162393162385%;*margin-left:62.28677941443899%;} .row-fluid .offset7:first-child{margin-left:59.82905982905982%;*margin-left:59.72267685033642%;} .row-fluid .offset6{margin-left:53.84615384615384%;*margin-left:53.739770867430444%;} .row-fluid .offset6:first-child{margin-left:51.28205128205128%;*margin-left:51.175668303327875%;} .row-fluid .offset5{margin-left:45.299145299145295%;*margin-left:45.1927623204219%;} .row-fluid .offset5:first-child{margin-left:42.73504273504273%;*margin-left:42.62865975631933%;} .row-fluid .offset4{margin-left:36.75213675213675%;*margin-left:36.645753773413354%;} .row-fluid .offset4:first-child{margin-left:34.18803418803419%;*margin-left:34.081651209310785%;} .row-fluid .offset3{margin-left:28.205128205128204%;*margin-left:28.0987452264048%;} .row-fluid .offset3:first-child{margin-left:25.641025641025642%;*margin-left:25.53464266230224%;} .row-fluid .offset2{margin-left:19.65811965811966%;*margin-left:19.551736679396257%;} .row-fluid .offset2:first-child{margin-left:17.094017094017094%;*margin-left:16.98763411529369%;} .row-fluid .offset1{margin-left:11.11111111111111%;*margin-left:11.004728132387708%;} .row-fluid .offset1:first-child{margin-left:8.547008547008547%;*margin-left:8.440625568285142%;} input,textarea,.uneditable-input{margin-left:0;} .controls-row [class*="span"]+[class*="span"]{margin-left:30px;} input.span12,textarea.span12,.uneditable-input.span12{width:1156px;} input.span11,textarea.span11,.uneditable-input.span11{width:1056px;} input.span10,textarea.span10,.uneditable-input.span10{width:956px;} input.span9,textarea.span9,.uneditable-input.span9{width:856px;} input.span8,textarea.span8,.uneditable-input.span8{width:756px;} input.span7,textarea.span7,.uneditable-input.span7{width:656px;} input.span6,textarea.span6,.uneditable-input.span6{width:556px;} input.span5,textarea.span5,.uneditable-input.span5{width:456px;} input.span4,textarea.span4,.uneditable-input.span4{width:356px;} input.span3,textarea.span3,.uneditable-input.span3{width:256px;} input.span2,textarea.span2,.uneditable-input.span2{width:156px;} input.span1,textarea.span1,.uneditable-input.span1{width:56px;} .thumbnails{margin-left:-30px;} .thumbnails>li{margin-left:30px;} .row-fluid .thumbnails{margin-left:0;}}@media (max-width:979px){body{padding-top:0;} .navbar-fixed-top,.navbar-fixed-bottom{position:static;} .navbar-fixed-top{margin-bottom:20px;} .navbar-fixed-bottom{margin-top:20px;} .navbar-fixed-top .navbar-inner,.navbar-fixed-bottom .navbar-inner{padding:5px;} .navbar .container{width:auto;padding:0;} .navbar .brand{padding-left:10px;padding-right:10px;margin:0 0 0 -5px;} .nav-collapse{clear:both;} .nav-collapse .nav{float:none;margin:0 0 10px;} .nav-collapse .nav>li{float:none;} .nav-collapse .nav>li>a{margin-bottom:2px;} .nav-collapse .nav>.divider-vertical{display:none;} .nav-collapse .nav .nav-header{color:#777777;text-shadow:none;} .nav-collapse .nav>li>a,.nav-collapse .dropdown-menu a{padding:9px 15px;font-weight:bold;color:#777777;-webkit-border-radius:3px;-moz-border-radius:3px;border-radius:3px;} .nav-collapse .btn{padding:4px 10px 4px;font-weight:normal;-webkit-border-radius:4px;-moz-border-radius:4px;border-radius:4px;} .nav-collapse .dropdown-menu li+li a{margin-bottom:2px;} .nav-collapse .nav>li>a:hover,.nav-collapse .nav>li>a:focus,.nav-collapse .dropdown-menu a:hover,.nav-collapse .dropdown-menu a:focus{background-color:#f2f2f2;} .navbar-inverse .nav-collapse .nav>li>a,.navbar-inverse .nav-collapse .dropdown-menu a{color:#999999;} .navbar-inverse .nav-collapse .nav>li>a:hover,.navbar-inverse .nav-collapse .nav>li>a:focus,.navbar-inverse .nav-collapse .dropdown-menu a:hover,.navbar-inverse .nav-collapse .dropdown-menu a:focus{background-color:#111111;} .nav-collapse.in .btn-group{margin-top:5px;padding:0;} .nav-collapse .dropdown-menu{position:static;top:auto;left:auto;float:none;display:none;max-width:none;margin:0 15px;padding:0;background-color:transparent;border:none;-webkit-border-radius:0;-moz-border-radius:0;border-radius:0;-webkit-box-shadow:none;-moz-box-shadow:none;box-shadow:none;} .nav-collapse .open>.dropdown-menu{display:block;} .nav-collapse .dropdown-menu:before,.nav-collapse .dropdown-menu:after{display:none;} .nav-collapse .dropdown-menu .divider{display:none;} .nav-collapse .nav>li>.dropdown-menu:before,.nav-collapse .nav>li>.dropdown-menu:after{display:none;} .nav-collapse .navbar-form,.nav-collapse .navbar-search{float:none;padding:10px 15px;margin:10px 0;border-top:1px solid #f2f2f2;border-bottom:1px solid #f2f2f2;-webkit-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);-moz-box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);box-shadow:inset 0 1px 0 rgba(255,255,255,.1), 0 1px 0 rgba(255,255,255,.1);} .navbar-inverse .nav-collapse .navbar-form,.navbar-inverse .nav-collapse .navbar-search{border-top-color:#111111;border-bottom-color:#111111;} .navbar .nav-collapse .nav.pull-right{float:none;margin-left:0;} .nav-collapse,.nav-collapse.collapse{overflow:hidden;height:0;} .navbar .btn-navbar{display:block;} .navbar-static .navbar-inner{padding-left:10px;padding-right:10px;}}@media (min-width:980px){.nav-collapse.collapse{height:auto !important;overflow:visible !important;}} diff --git a/dpark/web/ui/static/css/dag-viz.css b/dpark/web/ui/static/css/dag-viz.css new file mode 100644 index 00000000..9cc5c79f --- /dev/null +++ b/dpark/web/ui/static/css/dag-viz.css @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +#dag-viz-graph a, #dag-viz-graph a:hover { + text-decoration: none; +} + +#dag-viz-graph .label { + font-weight: normal; + text-shadow: none; +} + +#dag-viz-graph svg path { + stroke: #444; + stroke-width: 1.5px; +} + +#dag-viz-graph svg g.cluster rect { + stroke-width: 1px; +} + +#dag-viz-graph div#empty-dag-viz-message { + margin: 15px; +} + +/* Job page specific styles */ + +#dag-viz-graph svg.job marker#marker-arrow path { + fill: #333; + stroke-width: 0px; +} + +#dag-viz-graph svg.job g.cluster rect { + fill: #A0DFFF; + stroke: #3EC0FF; + stroke-width: 1px; +} + +#dag-viz-graph svg.job g.cluster.skipped rect { + fill: #D6D6D6; + stroke: #B7B7B7; + stroke-width: 1px; +} + +#dag-viz-graph svg.job g.cluster.stage rect { + fill: #FFFFFF; + stroke: #FF99AC; + stroke-width: 1px; +} + +#dag-viz-graph svg.job g.cluster.stage.skipped rect { + stroke: #ADADAD; + stroke-width: 1px; +} + +#dag-viz-graph svg.job g#cross-stage-edges path { + fill: none; +} + +#dag-viz-graph svg.job g.cluster text { + fill: #333; +} + +#dag-viz-graph svg.job g.cluster.skipped text { + fill: #666; +} + +#dag-viz-graph svg.job g.node circle { + fill: #444; +} + +#dag-viz-graph svg.job g.node.cached circle { + fill: #A3F545; + stroke: #52C366; + stroke-width: 2px; +} + +/* Stage page specific styles */ + +#dag-viz-graph svg.stage g.cluster rect { + fill: #A0DFFF; + stroke: #3EC0FF; + stroke-width: 1px; +} + +#dag-viz-graph svg.stage g.cluster.stage rect { + fill: #FFFFFF; + stroke: #FFA6B6; + stroke-width: 1px; +} + +#dag-viz-graph svg.stage g.node g.label text tspan { + fill: #333; +} + +#dag-viz-graph svg.stage g.cluster text { + fill: #333; +} + +#dag-viz-graph svg.stage g.node rect { + fill: #C3EBFF; + stroke: #3EC0FF; + stroke-width: 1px; +} + +#dag-viz-graph svg.stage g.node.cached rect { + fill: #B3F5C5; + stroke: #52C366; + stroke-width: 2px; +} + +.tooltip-inner { + white-space: pre-wrap; +} diff --git a/dpark/web/ui/static/js/bootstrap-tooltip.js b/dpark/web/ui/static/js/bootstrap-tooltip.js new file mode 100644 index 00000000..acd6096e --- /dev/null +++ b/dpark/web/ui/static/js/bootstrap-tooltip.js @@ -0,0 +1,361 @@ +/* =========================================================== + * bootstrap-tooltip.js v2.3.2 + * http://getbootstrap.com/2.3.2/javascript.html#tooltips + * Inspired by the original jQuery.tipsy by Jason Frame + * =========================================================== + * Copyright 2013 Twitter, Inc. + * + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + * ========================================================== */ + + +!function ($) { + + "use strict"; // jshint ;_; + + + /* TOOLTIP PUBLIC CLASS DEFINITION + * =============================== */ + + var Tooltip = function (element, options) { + this.init('tooltip', element, options) + } + + Tooltip.prototype = { + + constructor: Tooltip + + , init: function (type, element, options) { + var eventIn + , eventOut + , triggers + , trigger + , i + + this.type = type + this.$element = $(element) + this.options = this.getOptions(options) + this.enabled = true + + triggers = this.options.trigger.split(' ') + + for (i = triggers.length; i--;) { + trigger = triggers[i] + if (trigger == 'click') { + this.$element.on('click.' + this.type, this.options.selector, $.proxy(this.toggle, this)) + } else if (trigger != 'manual') { + eventIn = trigger == 'hover' ? 'mouseenter' : 'focus' + eventOut = trigger == 'hover' ? 'mouseleave' : 'blur' + this.$element.on(eventIn + '.' + this.type, this.options.selector, $.proxy(this.enter, this)) + this.$element.on(eventOut + '.' + this.type, this.options.selector, $.proxy(this.leave, this)) + } + } + + this.options.selector ? + (this._options = $.extend({}, this.options, { trigger: 'manual', selector: '' })) : + this.fixTitle() + } + + , getOptions: function (options) { + options = $.extend({}, $.fn[this.type].defaults, this.$element.data(), options) + + if (options.delay && typeof options.delay == 'number') { + options.delay = { + show: options.delay + , hide: options.delay + } + } + + return options + } + + , enter: function (e) { + var defaults = $.fn[this.type].defaults + , options = {} + , self + + this._options && $.each(this._options, function (key, value) { + if (defaults[key] != value) options[key] = value + }, this) + + self = $(e.currentTarget)[this.type](options).data(this.type) + + if (!self.options.delay || !self.options.delay.show) return self.show() + + clearTimeout(this.timeout) + self.hoverState = 'in' + this.timeout = setTimeout(function() { + if (self.hoverState == 'in') self.show() + }, self.options.delay.show) + } + + , leave: function (e) { + var self = $(e.currentTarget)[this.type](this._options).data(this.type) + + if (this.timeout) clearTimeout(this.timeout) + if (!self.options.delay || !self.options.delay.hide) return self.hide() + + self.hoverState = 'out' + this.timeout = setTimeout(function() { + if (self.hoverState == 'out') self.hide() + }, self.options.delay.hide) + } + + , show: function () { + var $tip + , pos + , actualWidth + , actualHeight + , placement + , tp + , e = $.Event('show') + + if (this.hasContent() && this.enabled) { + this.$element.trigger(e) + if (e.isDefaultPrevented()) return + $tip = this.tip() + this.setContent() + + if (this.options.animation) { + $tip.addClass('fade') + } + + placement = typeof this.options.placement == 'function' ? + this.options.placement.call(this, $tip[0], this.$element[0]) : + this.options.placement + + $tip + .detach() + .css({ top: 0, left: 0, display: 'block' }) + + this.options.container ? $tip.appendTo(this.options.container) : $tip.insertAfter(this.$element) + + pos = this.getPosition() + + actualWidth = $tip[0].offsetWidth + actualHeight = $tip[0].offsetHeight + + switch (placement) { + case 'bottom': + tp = {top: pos.top + pos.height, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'top': + tp = {top: pos.top - actualHeight, left: pos.left + pos.width / 2 - actualWidth / 2} + break + case 'left': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left - actualWidth} + break + case 'right': + tp = {top: pos.top + pos.height / 2 - actualHeight / 2, left: pos.left + pos.width} + break + } + + this.applyPlacement(tp, placement) + this.$element.trigger('shown') + } + } + + , applyPlacement: function(offset, placement){ + var $tip = this.tip() + , width = $tip[0].offsetWidth + , height = $tip[0].offsetHeight + , actualWidth + , actualHeight + , delta + , replace + + $tip + .offset(offset) + .addClass(placement) + .addClass('in') + + actualWidth = $tip[0].offsetWidth + actualHeight = $tip[0].offsetHeight + + if (placement == 'top' && actualHeight != height) { + offset.top = offset.top + height - actualHeight + replace = true + } + + if (placement == 'bottom' || placement == 'top') { + delta = 0 + + if (offset.left < 0){ + delta = offset.left * -2 + offset.left = 0 + $tip.offset(offset) + actualWidth = $tip[0].offsetWidth + actualHeight = $tip[0].offsetHeight + } + + this.replaceArrow(delta - width + actualWidth, actualWidth, 'left') + } else { + this.replaceArrow(actualHeight - height, actualHeight, 'top') + } + + if (replace) $tip.offset(offset) + } + + , replaceArrow: function(delta, dimension, position){ + this + .arrow() + .css(position, delta ? (50 * (1 - delta / dimension) + "%") : '') + } + + , setContent: function () { + var $tip = this.tip() + , title = this.getTitle() + + $tip.find('.tooltip-inner')[this.options.html ? 'html' : 'text'](title) + $tip.removeClass('fade in top bottom left right') + } + + , hide: function () { + var that = this + , $tip = this.tip() + , e = $.Event('hide') + + this.$element.trigger(e) + if (e.isDefaultPrevented()) return + + $tip.removeClass('in') + + function removeWithAnimation() { + var timeout = setTimeout(function () { + $tip.off($.support.transition.end).detach() + }, 500) + + $tip.one($.support.transition.end, function () { + clearTimeout(timeout) + $tip.detach() + }) + } + + $.support.transition && this.$tip.hasClass('fade') ? + removeWithAnimation() : + $tip.detach() + + this.$element.trigger('hidden') + + return this + } + + , fixTitle: function () { + var $e = this.$element + if ($e.attr('title') || typeof($e.attr('data-original-title')) != 'string') { + $e.attr('data-original-title', $e.attr('title') || '').attr('title', '') + } + } + + , hasContent: function () { + return this.getTitle() + } + + , getPosition: function () { + var el = this.$element[0] + return $.extend({}, (typeof el.getBoundingClientRect == 'function') ? el.getBoundingClientRect() : { + width: el.offsetWidth + , height: el.offsetHeight + }, this.$element.offset()) + } + + , getTitle: function () { + var title + , $e = this.$element + , o = this.options + + title = $e.attr('data-original-title') + || (typeof o.title == 'function' ? o.title.call($e[0]) : o.title) + + return title + } + + , tip: function () { + return this.$tip = this.$tip || $(this.options.template) + } + + , arrow: function(){ + return this.$arrow = this.$arrow || this.tip().find(".tooltip-arrow") + } + + , validate: function () { + if (!this.$element[0].parentNode) { + this.hide() + this.$element = null + this.options = null + } + } + + , enable: function () { + this.enabled = true + } + + , disable: function () { + this.enabled = false + } + + , toggleEnabled: function () { + this.enabled = !this.enabled + } + + , toggle: function (e) { + var self = e ? $(e.currentTarget)[this.type](this._options).data(this.type) : this + self.tip().hasClass('in') ? self.hide() : self.show() + } + + , destroy: function () { + this.hide().$element.off('.' + this.type).removeData(this.type) + } + + } + + + /* TOOLTIP PLUGIN DEFINITION + * ========================= */ + + var old = $.fn.tooltip + + $.fn.tooltip = function ( option ) { + return this.each(function () { + var $this = $(this) + , data = $this.data('tooltip') + , options = typeof option == 'object' && option + if (!data) $this.data('tooltip', (data = new Tooltip(this, options))) + if (typeof option == 'string') data[option]() + }) + } + + $.fn.tooltip.Constructor = Tooltip + + $.fn.tooltip.defaults = { + animation: true + , placement: 'top' + , selector: false + , template: '
' + , trigger: 'hover focus' + , title: '' + , delay: 0 + , html: false + , container: false + } + + + /* TOOLTIP NO CONFLICT + * =================== */ + + $.fn.tooltip.noConflict = function () { + $.fn.tooltip = old + return this + } + +}(window.jQuery); diff --git a/dpark/web/ui/static/js/d3.min.js b/dpark/web/ui/static/js/d3.min.js new file mode 100644 index 00000000..30cd2921 --- /dev/null +++ b/dpark/web/ui/static/js/d3.min.js @@ -0,0 +1,5 @@ +/*v3.5.5*/!function(){function n(n){return n&&(n.ownerDocument||n.document||n).documentElement}function t(n){return n&&(n.ownerDocument&&n.ownerDocument.defaultView||n.document&&n||n.defaultView)}function e(n,t){return t>n?-1:n>t?1:n>=t?0:0/0}function r(n){return null===n?0/0:+n}function u(n){return!isNaN(n)}function i(n){return{left:function(t,e,r,u){for(arguments.length<3&&(r=0),arguments.length<4&&(u=t.length);u>r;){var i=r+u>>>1;n(t[i],e)<0?r=i+1:u=i}return r},right:function(t,e,r,u){for(arguments.length<3&&(r=0),arguments.length<4&&(u=t.length);u>r;){var i=r+u>>>1;n(t[i],e)>0?u=i:r=i+1}return r}}}function o(n){return n.length}function a(n){for(var t=1;n*t%1;)t*=10;return t}function c(n,t){for(var e in t)Object.defineProperty(n.prototype,e,{value:t[e],enumerable:!1})}function l(){this._=Object.create(null)}function s(n){return(n+="")===pa||n[0]===va?va+n:n}function f(n){return(n+="")[0]===va?n.slice(1):n}function h(n){return s(n)in this._}function g(n){return(n=s(n))in this._&&delete this._[n]}function p(){var n=[];for(var t in this._)n.push(f(t));return n}function v(){var n=0;for(var t in this._)++n;return n}function d(){for(var n in this._)return!1;return!0}function m(){this._=Object.create(null)}function y(n){return n}function M(n,t,e){return function(){var r=e.apply(t,arguments);return r===t?n:r}}function x(n,t){if(t in n)return t;t=t.charAt(0).toUpperCase()+t.slice(1);for(var e=0,r=da.length;r>e;++e){var u=da[e]+t;if(u in n)return u}}function b(){}function _(){}function w(n){function t(){for(var t,r=e,u=-1,i=r.length;++ue;e++)for(var u,i=n[e],o=0,a=i.length;a>o;o++)(u=i[o])&&t(u,o,e);return n}function Z(n){return ya(n,Sa),n}function V(n){var t,e;return function(r,u,i){var o,a=n[i].update,c=a.length;for(i!=e&&(e=i,t=0),u>=t&&(t=u+1);!(o=a[t])&&++t0&&(n=n.slice(0,a));var l=ka.get(n);return l&&(n=l,c=B),a?t?u:r:t?b:i}function $(n,t){return function(e){var r=ta.event;ta.event=e,t[0]=this.__data__;try{n.apply(this,t)}finally{ta.event=r}}}function B(n,t){var e=$(n,t);return function(n){var t=this,r=n.relatedTarget;r&&(r===t||8&r.compareDocumentPosition(t))||e.call(t,n)}}function W(e){var r=".dragsuppress-"+ ++Aa,u="click"+r,i=ta.select(t(e)).on("touchmove"+r,S).on("dragstart"+r,S).on("selectstart"+r,S);if(null==Ea&&(Ea="onselectstart"in e?!1:x(e.style,"userSelect")),Ea){var o=n(e).style,a=o[Ea];o[Ea]="none"}return function(n){if(i.on(r,null),Ea&&(o[Ea]=a),n){var t=function(){i.on(u,null)};i.on(u,function(){S(),t()},!0),setTimeout(t,0)}}}function J(n,e){e.changedTouches&&(e=e.changedTouches[0]);var r=n.ownerSVGElement||n;if(r.createSVGPoint){var u=r.createSVGPoint();if(0>Na){var i=t(n);if(i.scrollX||i.scrollY){r=ta.select("body").append("svg").style({position:"absolute",top:0,left:0,margin:0,padding:0,border:"none"},"important");var o=r[0][0].getScreenCTM();Na=!(o.f||o.e),r.remove()}}return Na?(u.x=e.pageX,u.y=e.pageY):(u.x=e.clientX,u.y=e.clientY),u=u.matrixTransform(n.getScreenCTM().inverse()),[u.x,u.y]}var a=n.getBoundingClientRect();return[e.clientX-a.left-n.clientLeft,e.clientY-a.top-n.clientTop]}function G(){return ta.event.changedTouches[0].identifier}function K(n){return n>0?1:0>n?-1:0}function Q(n,t,e){return(t[0]-n[0])*(e[1]-n[1])-(t[1]-n[1])*(e[0]-n[0])}function nt(n){return n>1?0:-1>n?qa:Math.acos(n)}function tt(n){return n>1?Ra:-1>n?-Ra:Math.asin(n)}function et(n){return((n=Math.exp(n))-1/n)/2}function rt(n){return((n=Math.exp(n))+1/n)/2}function ut(n){return((n=Math.exp(2*n))-1)/(n+1)}function it(n){return(n=Math.sin(n/2))*n}function ot(){}function at(n,t,e){return this instanceof at?(this.h=+n,this.s=+t,void(this.l=+e)):arguments.length<2?n instanceof at?new at(n.h,n.s,n.l):bt(""+n,_t,at):new at(n,t,e)}function ct(n,t,e){function r(n){return n>360?n-=360:0>n&&(n+=360),60>n?i+(o-i)*n/60:180>n?o:240>n?i+(o-i)*(240-n)/60:i}function u(n){return Math.round(255*r(n))}var i,o;return n=isNaN(n)?0:(n%=360)<0?n+360:n,t=isNaN(t)?0:0>t?0:t>1?1:t,e=0>e?0:e>1?1:e,o=.5>=e?e*(1+t):e+t-e*t,i=2*e-o,new mt(u(n+120),u(n),u(n-120))}function lt(n,t,e){return this instanceof lt?(this.h=+n,this.c=+t,void(this.l=+e)):arguments.length<2?n instanceof lt?new lt(n.h,n.c,n.l):n instanceof ft?gt(n.l,n.a,n.b):gt((n=wt((n=ta.rgb(n)).r,n.g,n.b)).l,n.a,n.b):new lt(n,t,e)}function st(n,t,e){return isNaN(n)&&(n=0),isNaN(t)&&(t=0),new ft(e,Math.cos(n*=Da)*t,Math.sin(n)*t)}function ft(n,t,e){return this instanceof ft?(this.l=+n,this.a=+t,void(this.b=+e)):arguments.length<2?n instanceof ft?new ft(n.l,n.a,n.b):n instanceof lt?st(n.h,n.c,n.l):wt((n=mt(n)).r,n.g,n.b):new ft(n,t,e)}function ht(n,t,e){var r=(n+16)/116,u=r+t/500,i=r-e/200;return u=pt(u)*Xa,r=pt(r)*$a,i=pt(i)*Ba,new mt(dt(3.2404542*u-1.5371385*r-.4985314*i),dt(-.969266*u+1.8760108*r+.041556*i),dt(.0556434*u-.2040259*r+1.0572252*i))}function gt(n,t,e){return n>0?new lt(Math.atan2(e,t)*Pa,Math.sqrt(t*t+e*e),n):new lt(0/0,0/0,n)}function pt(n){return n>.206893034?n*n*n:(n-4/29)/7.787037}function vt(n){return n>.008856?Math.pow(n,1/3):7.787037*n+4/29}function dt(n){return Math.round(255*(.00304>=n?12.92*n:1.055*Math.pow(n,1/2.4)-.055))}function mt(n,t,e){return this instanceof mt?(this.r=~~n,this.g=~~t,void(this.b=~~e)):arguments.length<2?n instanceof mt?new mt(n.r,n.g,n.b):bt(""+n,mt,ct):new mt(n,t,e)}function yt(n){return new mt(n>>16,n>>8&255,255&n)}function Mt(n){return yt(n)+""}function xt(n){return 16>n?"0"+Math.max(0,n).toString(16):Math.min(255,n).toString(16)}function bt(n,t,e){var r,u,i,o=0,a=0,c=0;if(r=/([a-z]+)\((.*)\)/i.exec(n))switch(u=r[2].split(","),r[1]){case"hsl":return e(parseFloat(u[0]),parseFloat(u[1])/100,parseFloat(u[2])/100);case"rgb":return t(kt(u[0]),kt(u[1]),kt(u[2]))}return(i=Ga.get(n.toLowerCase()))?t(i.r,i.g,i.b):(null==n||"#"!==n.charAt(0)||isNaN(i=parseInt(n.slice(1),16))||(4===n.length?(o=(3840&i)>>4,o=o>>4|o,a=240&i,a=a>>4|a,c=15&i,c=c<<4|c):7===n.length&&(o=(16711680&i)>>16,a=(65280&i)>>8,c=255&i)),t(o,a,c))}function _t(n,t,e){var r,u,i=Math.min(n/=255,t/=255,e/=255),o=Math.max(n,t,e),a=o-i,c=(o+i)/2;return a?(u=.5>c?a/(o+i):a/(2-o-i),r=n==o?(t-e)/a+(e>t?6:0):t==o?(e-n)/a+2:(n-t)/a+4,r*=60):(r=0/0,u=c>0&&1>c?0:r),new at(r,u,c)}function wt(n,t,e){n=St(n),t=St(t),e=St(e);var r=vt((.4124564*n+.3575761*t+.1804375*e)/Xa),u=vt((.2126729*n+.7151522*t+.072175*e)/$a),i=vt((.0193339*n+.119192*t+.9503041*e)/Ba);return ft(116*u-16,500*(r-u),200*(u-i))}function St(n){return(n/=255)<=.04045?n/12.92:Math.pow((n+.055)/1.055,2.4)}function kt(n){var t=parseFloat(n);return"%"===n.charAt(n.length-1)?Math.round(2.55*t):t}function Et(n){return"function"==typeof n?n:function(){return n}}function At(n){return function(t,e,r){return 2===arguments.length&&"function"==typeof e&&(r=e,e=null),Nt(t,e,n,r)}}function Nt(n,t,e,r){function u(){var n,t=c.status;if(!t&&zt(c)||t>=200&&300>t||304===t){try{n=e.call(i,c)}catch(r){return void o.error.call(i,r)}o.load.call(i,n)}else o.error.call(i,c)}var i={},o=ta.dispatch("beforesend","progress","load","error"),a={},c=new XMLHttpRequest,l=null;return!this.XDomainRequest||"withCredentials"in c||!/^(http(s)?:)?\/\//.test(n)||(c=new XDomainRequest),"onload"in c?c.onload=c.onerror=u:c.onreadystatechange=function(){c.readyState>3&&u()},c.onprogress=function(n){var t=ta.event;ta.event=n;try{o.progress.call(i,c)}finally{ta.event=t}},i.header=function(n,t){return n=(n+"").toLowerCase(),arguments.length<2?a[n]:(null==t?delete a[n]:a[n]=t+"",i)},i.mimeType=function(n){return arguments.length?(t=null==n?null:n+"",i):t},i.responseType=function(n){return arguments.length?(l=n,i):l},i.response=function(n){return e=n,i},["get","post"].forEach(function(n){i[n]=function(){return i.send.apply(i,[n].concat(ra(arguments)))}}),i.send=function(e,r,u){if(2===arguments.length&&"function"==typeof r&&(u=r,r=null),c.open(e,n,!0),null==t||"accept"in a||(a.accept=t+",*/*"),c.setRequestHeader)for(var s in a)c.setRequestHeader(s,a[s]);return null!=t&&c.overrideMimeType&&c.overrideMimeType(t),null!=l&&(c.responseType=l),null!=u&&i.on("error",u).on("load",function(n){u(null,n)}),o.beforesend.call(i,c),c.send(null==r?null:r),i},i.abort=function(){return c.abort(),i},ta.rebind(i,o,"on"),null==r?i:i.get(Ct(r))}function Ct(n){return 1===n.length?function(t,e){n(null==t?e:null)}:n}function zt(n){var t=n.responseType;return t&&"text"!==t?n.response:n.responseText}function qt(){var n=Lt(),t=Tt()-n;t>24?(isFinite(t)&&(clearTimeout(tc),tc=setTimeout(qt,t)),nc=0):(nc=1,rc(qt))}function Lt(){var n=Date.now();for(ec=Ka;ec;)n>=ec.t&&(ec.f=ec.c(n-ec.t)),ec=ec.n;return n}function Tt(){for(var n,t=Ka,e=1/0;t;)t.f?t=n?n.n=t.n:Ka=t.n:(t.t8?function(n){return n/e}:function(n){return n*e},symbol:n}}function Pt(n){var t=n.decimal,e=n.thousands,r=n.grouping,u=n.currency,i=r&&e?function(n,t){for(var u=n.length,i=[],o=0,a=r[0],c=0;u>0&&a>0&&(c+a+1>t&&(a=Math.max(1,t-c)),i.push(n.substring(u-=a,u+a)),!((c+=a+1)>t));)a=r[o=(o+1)%r.length];return i.reverse().join(e)}:y;return function(n){var e=ic.exec(n),r=e[1]||" ",o=e[2]||">",a=e[3]||"-",c=e[4]||"",l=e[5],s=+e[6],f=e[7],h=e[8],g=e[9],p=1,v="",d="",m=!1,y=!0;switch(h&&(h=+h.substring(1)),(l||"0"===r&&"="===o)&&(l=r="0",o="="),g){case"n":f=!0,g="g";break;case"%":p=100,d="%",g="f";break;case"p":p=100,d="%",g="r";break;case"b":case"o":case"x":case"X":"#"===c&&(v="0"+g.toLowerCase());case"c":y=!1;case"d":m=!0,h=0;break;case"s":p=-1,g="r"}"$"===c&&(v=u[0],d=u[1]),"r"!=g||h||(g="g"),null!=h&&("g"==g?h=Math.max(1,Math.min(21,h)):("e"==g||"f"==g)&&(h=Math.max(0,Math.min(20,h)))),g=oc.get(g)||Ut;var M=l&&f;return function(n){var e=d;if(m&&n%1)return"";var u=0>n||0===n&&0>1/n?(n=-n,"-"):"-"===a?"":a;if(0>p){var c=ta.formatPrefix(n,h);n=c.scale(n),e=c.symbol+d}else n*=p;n=g(n,h);var x,b,_=n.lastIndexOf(".");if(0>_){var w=y?n.lastIndexOf("e"):-1;0>w?(x=n,b=""):(x=n.substring(0,w),b=n.substring(w))}else x=n.substring(0,_),b=t+n.substring(_+1);!l&&f&&(x=i(x,1/0));var S=v.length+x.length+b.length+(M?0:u.length),k=s>S?new Array(S=s-S+1).join(r):"";return M&&(x=i(k+x,k.length?s-b.length:1/0)),u+=v,n=x+b,("<"===o?u+n+k:">"===o?k+u+n:"^"===o?k.substring(0,S>>=1)+u+n+k.substring(S):u+(M?n:k+n))+e}}}function Ut(n){return n+""}function jt(){this._=new Date(arguments.length>1?Date.UTC.apply(this,arguments):arguments[0])}function Ft(n,t,e){function r(t){var e=n(t),r=i(e,1);return r-t>t-e?e:r}function u(e){return t(e=n(new cc(e-1)),1),e}function i(n,e){return t(n=new cc(+n),e),n}function o(n,r,i){var o=u(n),a=[];if(i>1)for(;r>o;)e(o)%i||a.push(new Date(+o)),t(o,1);else for(;r>o;)a.push(new Date(+o)),t(o,1);return a}function a(n,t,e){try{cc=jt;var r=new jt;return r._=n,o(r,t,e)}finally{cc=Date}}n.floor=n,n.round=r,n.ceil=u,n.offset=i,n.range=o;var c=n.utc=Ht(n);return c.floor=c,c.round=Ht(r),c.ceil=Ht(u),c.offset=Ht(i),c.range=a,n}function Ht(n){return function(t,e){try{cc=jt;var r=new jt;return r._=t,n(r,e)._}finally{cc=Date}}}function Ot(n){function t(n){function t(t){for(var e,u,i,o=[],a=-1,c=0;++aa;){if(r>=l)return-1;if(u=t.charCodeAt(a++),37===u){if(o=t.charAt(a++),i=C[o in sc?t.charAt(a++):o],!i||(r=i(n,e,r))<0)return-1}else if(u!=e.charCodeAt(r++))return-1}return r}function r(n,t,e){_.lastIndex=0;var r=_.exec(t.slice(e));return r?(n.w=w.get(r[0].toLowerCase()),e+r[0].length):-1}function u(n,t,e){x.lastIndex=0;var r=x.exec(t.slice(e));return r?(n.w=b.get(r[0].toLowerCase()),e+r[0].length):-1}function i(n,t,e){E.lastIndex=0;var r=E.exec(t.slice(e));return r?(n.m=A.get(r[0].toLowerCase()),e+r[0].length):-1}function o(n,t,e){S.lastIndex=0;var r=S.exec(t.slice(e));return r?(n.m=k.get(r[0].toLowerCase()),e+r[0].length):-1}function a(n,t,r){return e(n,N.c.toString(),t,r)}function c(n,t,r){return e(n,N.x.toString(),t,r)}function l(n,t,r){return e(n,N.X.toString(),t,r)}function s(n,t,e){var r=M.get(t.slice(e,e+=2).toLowerCase());return null==r?-1:(n.p=r,e)}var f=n.dateTime,h=n.date,g=n.time,p=n.periods,v=n.days,d=n.shortDays,m=n.months,y=n.shortMonths;t.utc=function(n){function e(n){try{cc=jt;var t=new cc;return t._=n,r(t)}finally{cc=Date}}var r=t(n);return e.parse=function(n){try{cc=jt;var t=r.parse(n);return t&&t._}finally{cc=Date}},e.toString=r.toString,e},t.multi=t.utc.multi=ae;var M=ta.map(),x=Yt(v),b=Zt(v),_=Yt(d),w=Zt(d),S=Yt(m),k=Zt(m),E=Yt(y),A=Zt(y);p.forEach(function(n,t){M.set(n.toLowerCase(),t)});var N={a:function(n){return d[n.getDay()]},A:function(n){return v[n.getDay()]},b:function(n){return y[n.getMonth()]},B:function(n){return m[n.getMonth()]},c:t(f),d:function(n,t){return It(n.getDate(),t,2)},e:function(n,t){return It(n.getDate(),t,2)},H:function(n,t){return It(n.getHours(),t,2)},I:function(n,t){return It(n.getHours()%12||12,t,2)},j:function(n,t){return It(1+ac.dayOfYear(n),t,3)},L:function(n,t){return It(n.getMilliseconds(),t,3)},m:function(n,t){return It(n.getMonth()+1,t,2)},M:function(n,t){return It(n.getMinutes(),t,2)},p:function(n){return p[+(n.getHours()>=12)]},S:function(n,t){return It(n.getSeconds(),t,2)},U:function(n,t){return It(ac.sundayOfYear(n),t,2)},w:function(n){return n.getDay()},W:function(n,t){return It(ac.mondayOfYear(n),t,2)},x:t(h),X:t(g),y:function(n,t){return It(n.getFullYear()%100,t,2)},Y:function(n,t){return It(n.getFullYear()%1e4,t,4)},Z:ie,"%":function(){return"%"}},C={a:r,A:u,b:i,B:o,c:a,d:Qt,e:Qt,H:te,I:te,j:ne,L:ue,m:Kt,M:ee,p:s,S:re,U:Xt,w:Vt,W:$t,x:c,X:l,y:Wt,Y:Bt,Z:Jt,"%":oe};return t}function It(n,t,e){var r=0>n?"-":"",u=(r?-n:n)+"",i=u.length;return r+(e>i?new Array(e-i+1).join(t)+u:u)}function Yt(n){return new RegExp("^(?:"+n.map(ta.requote).join("|")+")","i")}function Zt(n){for(var t=new l,e=-1,r=n.length;++e68?1900:2e3)}function Kt(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.m=r[0]-1,e+r[0].length):-1}function Qt(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.d=+r[0],e+r[0].length):-1}function ne(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+3));return r?(n.j=+r[0],e+r[0].length):-1}function te(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.H=+r[0],e+r[0].length):-1}function ee(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.M=+r[0],e+r[0].length):-1}function re(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+2));return r?(n.S=+r[0],e+r[0].length):-1}function ue(n,t,e){fc.lastIndex=0;var r=fc.exec(t.slice(e,e+3));return r?(n.L=+r[0],e+r[0].length):-1}function ie(n){var t=n.getTimezoneOffset(),e=t>0?"-":"+",r=ga(t)/60|0,u=ga(t)%60;return e+It(r,"0",2)+It(u,"0",2)}function oe(n,t,e){hc.lastIndex=0;var r=hc.exec(t.slice(e,e+1));return r?e+r[0].length:-1}function ae(n){for(var t=n.length,e=-1;++e=0?1:-1,a=o*e,c=Math.cos(t),l=Math.sin(t),s=i*l,f=u*c+s*Math.cos(a),h=s*o*Math.sin(a);yc.add(Math.atan2(h,f)),r=n,u=c,i=l}var t,e,r,u,i;Mc.point=function(o,a){Mc.point=n,r=(t=o)*Da,u=Math.cos(a=(e=a)*Da/2+qa/4),i=Math.sin(a)},Mc.lineEnd=function(){n(t,e)}}function pe(n){var t=n[0],e=n[1],r=Math.cos(e);return[r*Math.cos(t),r*Math.sin(t),Math.sin(e)]}function ve(n,t){return n[0]*t[0]+n[1]*t[1]+n[2]*t[2]}function de(n,t){return[n[1]*t[2]-n[2]*t[1],n[2]*t[0]-n[0]*t[2],n[0]*t[1]-n[1]*t[0]]}function me(n,t){n[0]+=t[0],n[1]+=t[1],n[2]+=t[2]}function ye(n,t){return[n[0]*t,n[1]*t,n[2]*t]}function Me(n){var t=Math.sqrt(n[0]*n[0]+n[1]*n[1]+n[2]*n[2]);n[0]/=t,n[1]/=t,n[2]/=t}function xe(n){return[Math.atan2(n[1],n[0]),tt(n[2])]}function be(n,t){return ga(n[0]-t[0])a;++a)u.point((e=n[a])[0],e[1]);return void u.lineEnd()}var c=new qe(e,n,null,!0),l=new qe(e,null,c,!1);c.o=l,i.push(c),o.push(l),c=new qe(r,n,null,!1),l=new qe(r,null,c,!0),c.o=l,i.push(c),o.push(l)}}),o.sort(t),ze(i),ze(o),i.length){for(var a=0,c=e,l=o.length;l>a;++a)o[a].e=c=!c;for(var s,f,h=i[0];;){for(var g=h,p=!0;g.v;)if((g=g.n)===h)return;s=g.z,u.lineStart();do{if(g.v=g.o.v=!0,g.e){if(p)for(var a=0,l=s.length;l>a;++a)u.point((f=s[a])[0],f[1]);else r(g.x,g.n.x,1,u);g=g.n}else{if(p){s=g.p.z;for(var a=s.length-1;a>=0;--a)u.point((f=s[a])[0],f[1])}else r(g.x,g.p.x,-1,u);g=g.p}g=g.o,s=g.z,p=!p}while(!g.v);u.lineEnd()}}}function ze(n){if(t=n.length){for(var t,e,r=0,u=n[0];++r0){for(b||(i.polygonStart(),b=!0),i.lineStart();++o1&&2&t&&e.push(e.pop().concat(e.shift())),g.push(e.filter(Te))}var g,p,v,d=t(i),m=u.invert(r[0],r[1]),y={point:o,lineStart:c,lineEnd:l,polygonStart:function(){y.point=s,y.lineStart=f,y.lineEnd=h,g=[],p=[]},polygonEnd:function(){y.point=o,y.lineStart=c,y.lineEnd=l,g=ta.merge(g);var n=Fe(m,p);g.length?(b||(i.polygonStart(),b=!0),Ce(g,De,n,e,i)):n&&(b||(i.polygonStart(),b=!0),i.lineStart(),e(null,null,1,i),i.lineEnd()),b&&(i.polygonEnd(),b=!1),g=p=null},sphere:function(){i.polygonStart(),i.lineStart(),e(null,null,1,i),i.lineEnd(),i.polygonEnd()}},M=Re(),x=t(M),b=!1;return y}}function Te(n){return n.length>1}function Re(){var n,t=[];return{lineStart:function(){t.push(n=[])},point:function(t,e){n.push([t,e])},lineEnd:b,buffer:function(){var e=t;return t=[],n=null,e},rejoin:function(){t.length>1&&t.push(t.pop().concat(t.shift()))}}}function De(n,t){return((n=n.x)[0]<0?n[1]-Ra-Ca:Ra-n[1])-((t=t.x)[0]<0?t[1]-Ra-Ca:Ra-t[1])}function Pe(n){var t,e=0/0,r=0/0,u=0/0;return{lineStart:function(){n.lineStart(),t=1},point:function(i,o){var a=i>0?qa:-qa,c=ga(i-e);ga(c-qa)0?Ra:-Ra),n.point(u,r),n.lineEnd(),n.lineStart(),n.point(a,r),n.point(i,r),t=0):u!==a&&c>=qa&&(ga(e-u)Ca?Math.atan((Math.sin(t)*(i=Math.cos(r))*Math.sin(e)-Math.sin(r)*(u=Math.cos(t))*Math.sin(n))/(u*i*o)):(t+r)/2}function je(n,t,e,r){var u;if(null==n)u=e*Ra,r.point(-qa,u),r.point(0,u),r.point(qa,u),r.point(qa,0),r.point(qa,-u),r.point(0,-u),r.point(-qa,-u),r.point(-qa,0),r.point(-qa,u);else if(ga(n[0]-t[0])>Ca){var i=n[0]a;++a){var l=t[a],s=l.length;if(s)for(var f=l[0],h=f[0],g=f[1]/2+qa/4,p=Math.sin(g),v=Math.cos(g),d=1;;){d===s&&(d=0),n=l[d];var m=n[0],y=n[1]/2+qa/4,M=Math.sin(y),x=Math.cos(y),b=m-h,_=b>=0?1:-1,w=_*b,S=w>qa,k=p*M;if(yc.add(Math.atan2(k*_*Math.sin(w),v*x+k*Math.cos(w))),i+=S?b+_*La:b,S^h>=e^m>=e){var E=de(pe(f),pe(n));Me(E);var A=de(u,E);Me(A);var N=(S^b>=0?-1:1)*tt(A[2]);(r>N||r===N&&(E[0]||E[1]))&&(o+=S^b>=0?1:-1)}if(!d++)break;h=m,p=M,v=x,f=n}}return(-Ca>i||Ca>i&&0>yc)^1&o}function He(n){function t(n,t){return Math.cos(n)*Math.cos(t)>i}function e(n){var e,i,c,l,s;return{lineStart:function(){l=c=!1,s=1},point:function(f,h){var g,p=[f,h],v=t(f,h),d=o?v?0:u(f,h):v?u(f+(0>f?qa:-qa),h):0;if(!e&&(l=c=v)&&n.lineStart(),v!==c&&(g=r(e,p),(be(e,g)||be(p,g))&&(p[0]+=Ca,p[1]+=Ca,v=t(p[0],p[1]))),v!==c)s=0,v?(n.lineStart(),g=r(p,e),n.point(g[0],g[1])):(g=r(e,p),n.point(g[0],g[1]),n.lineEnd()),e=g;else if(a&&e&&o^v){var m;d&i||!(m=r(p,e,!0))||(s=0,o?(n.lineStart(),n.point(m[0][0],m[0][1]),n.point(m[1][0],m[1][1]),n.lineEnd()):(n.point(m[1][0],m[1][1]),n.lineEnd(),n.lineStart(),n.point(m[0][0],m[0][1])))}!v||e&&be(e,p)||n.point(p[0],p[1]),e=p,c=v,i=d},lineEnd:function(){c&&n.lineEnd(),e=null},clean:function(){return s|(l&&c)<<1}}}function r(n,t,e){var r=pe(n),u=pe(t),o=[1,0,0],a=de(r,u),c=ve(a,a),l=a[0],s=c-l*l;if(!s)return!e&&n;var f=i*c/s,h=-i*l/s,g=de(o,a),p=ye(o,f),v=ye(a,h);me(p,v);var d=g,m=ve(p,d),y=ve(d,d),M=m*m-y*(ve(p,p)-1);if(!(0>M)){var x=Math.sqrt(M),b=ye(d,(-m-x)/y);if(me(b,p),b=xe(b),!e)return b;var _,w=n[0],S=t[0],k=n[1],E=t[1];w>S&&(_=w,w=S,S=_);var A=S-w,N=ga(A-qa)A;if(!N&&k>E&&(_=k,k=E,E=_),C?N?k+E>0^b[1]<(ga(b[0]-w)qa^(w<=b[0]&&b[0]<=S)){var z=ye(d,(-m+x)/y);return me(z,p),[b,xe(z)]}}}function u(t,e){var r=o?n:qa-n,u=0;return-r>t?u|=1:t>r&&(u|=2),-r>e?u|=4:e>r&&(u|=8),u}var i=Math.cos(n),o=i>0,a=ga(i)>Ca,c=gr(n,6*Da);return Le(t,e,c,o?[0,-n]:[-qa,n-qa])}function Oe(n,t,e,r){return function(u){var i,o=u.a,a=u.b,c=o.x,l=o.y,s=a.x,f=a.y,h=0,g=1,p=s-c,v=f-l;if(i=n-c,p||!(i>0)){if(i/=p,0>p){if(h>i)return;g>i&&(g=i)}else if(p>0){if(i>g)return;i>h&&(h=i)}if(i=e-c,p||!(0>i)){if(i/=p,0>p){if(i>g)return;i>h&&(h=i)}else if(p>0){if(h>i)return;g>i&&(g=i)}if(i=t-l,v||!(i>0)){if(i/=v,0>v){if(h>i)return;g>i&&(g=i)}else if(v>0){if(i>g)return;i>h&&(h=i)}if(i=r-l,v||!(0>i)){if(i/=v,0>v){if(i>g)return;i>h&&(h=i)}else if(v>0){if(h>i)return;g>i&&(g=i)}return h>0&&(u.a={x:c+h*p,y:l+h*v}),1>g&&(u.b={x:c+g*p,y:l+g*v}),u}}}}}}function Ie(n,t,e,r){function u(r,u){return ga(r[0]-n)0?0:3:ga(r[0]-e)0?2:1:ga(r[1]-t)0?1:0:u>0?3:2}function i(n,t){return o(n.x,t.x)}function o(n,t){var e=u(n,1),r=u(t,1);return e!==r?e-r:0===e?t[1]-n[1]:1===e?n[0]-t[0]:2===e?n[1]-t[1]:t[0]-n[0]}return function(a){function c(n){for(var t=0,e=d.length,r=n[1],u=0;e>u;++u)for(var i,o=1,a=d[u],c=a.length,l=a[0];c>o;++o)i=a[o],l[1]<=r?i[1]>r&&Q(l,i,n)>0&&++t:i[1]<=r&&Q(l,i,n)<0&&--t,l=i;return 0!==t}function l(i,a,c,l){var s=0,f=0;if(null==i||(s=u(i,c))!==(f=u(a,c))||o(i,a)<0^c>0){do l.point(0===s||3===s?n:e,s>1?r:t);while((s=(s+c+4)%4)!==f)}else l.point(a[0],a[1])}function s(u,i){return u>=n&&e>=u&&i>=t&&r>=i}function f(n,t){s(n,t)&&a.point(n,t)}function h(){C.point=p,d&&d.push(m=[]),S=!0,w=!1,b=_=0/0}function g(){v&&(p(y,M),x&&w&&A.rejoin(),v.push(A.buffer())),C.point=f,w&&a.lineEnd()}function p(n,t){n=Math.max(-Tc,Math.min(Tc,n)),t=Math.max(-Tc,Math.min(Tc,t));var e=s(n,t);if(d&&m.push([n,t]),S)y=n,M=t,x=e,S=!1,e&&(a.lineStart(),a.point(n,t));else if(e&&w)a.point(n,t);else{var r={a:{x:b,y:_},b:{x:n,y:t}};N(r)?(w||(a.lineStart(),a.point(r.a.x,r.a.y)),a.point(r.b.x,r.b.y),e||a.lineEnd(),k=!1):e&&(a.lineStart(),a.point(n,t),k=!1)}b=n,_=t,w=e}var v,d,m,y,M,x,b,_,w,S,k,E=a,A=Re(),N=Oe(n,t,e,r),C={point:f,lineStart:h,lineEnd:g,polygonStart:function(){a=A,v=[],d=[],k=!0},polygonEnd:function(){a=E,v=ta.merge(v);var t=c([n,r]),e=k&&t,u=v.length;(e||u)&&(a.polygonStart(),e&&(a.lineStart(),l(null,null,1,a),a.lineEnd()),u&&Ce(v,i,t,l,a),a.polygonEnd()),v=d=m=null}};return C}}function Ye(n){var t=0,e=qa/3,r=ir(n),u=r(t,e);return u.parallels=function(n){return arguments.length?r(t=n[0]*qa/180,e=n[1]*qa/180):[t/qa*180,e/qa*180]},u}function Ze(n,t){function e(n,t){var e=Math.sqrt(i-2*u*Math.sin(t))/u;return[e*Math.sin(n*=u),o-e*Math.cos(n)]}var r=Math.sin(n),u=(r+Math.sin(t))/2,i=1+r*(2*u-r),o=Math.sqrt(i)/u;return e.invert=function(n,t){var e=o-t;return[Math.atan2(n,e)/u,tt((i-(n*n+e*e)*u*u)/(2*u))]},e}function Ve(){function n(n,t){Dc+=u*n-r*t,r=n,u=t}var t,e,r,u;Hc.point=function(i,o){Hc.point=n,t=r=i,e=u=o},Hc.lineEnd=function(){n(t,e)}}function Xe(n,t){Pc>n&&(Pc=n),n>jc&&(jc=n),Uc>t&&(Uc=t),t>Fc&&(Fc=t)}function $e(){function n(n,t){o.push("M",n,",",t,i)}function t(n,t){o.push("M",n,",",t),a.point=e}function e(n,t){o.push("L",n,",",t)}function r(){a.point=n}function u(){o.push("Z")}var i=Be(4.5),o=[],a={point:n,lineStart:function(){a.point=t},lineEnd:r,polygonStart:function(){a.lineEnd=u},polygonEnd:function(){a.lineEnd=r,a.point=n},pointRadius:function(n){return i=Be(n),a},result:function(){if(o.length){var n=o.join("");return o=[],n}}};return a}function Be(n){return"m0,"+n+"a"+n+","+n+" 0 1,1 0,"+-2*n+"a"+n+","+n+" 0 1,1 0,"+2*n+"z"}function We(n,t){_c+=n,wc+=t,++Sc}function Je(){function n(n,r){var u=n-t,i=r-e,o=Math.sqrt(u*u+i*i);kc+=o*(t+n)/2,Ec+=o*(e+r)/2,Ac+=o,We(t=n,e=r)}var t,e;Ic.point=function(r,u){Ic.point=n,We(t=r,e=u)}}function Ge(){Ic.point=We}function Ke(){function n(n,t){var e=n-r,i=t-u,o=Math.sqrt(e*e+i*i);kc+=o*(r+n)/2,Ec+=o*(u+t)/2,Ac+=o,o=u*n-r*t,Nc+=o*(r+n),Cc+=o*(u+t),zc+=3*o,We(r=n,u=t)}var t,e,r,u;Ic.point=function(i,o){Ic.point=n,We(t=r=i,e=u=o)},Ic.lineEnd=function(){n(t,e)}}function Qe(n){function t(t,e){n.moveTo(t+o,e),n.arc(t,e,o,0,La)}function e(t,e){n.moveTo(t,e),a.point=r}function r(t,e){n.lineTo(t,e)}function u(){a.point=t}function i(){n.closePath()}var o=4.5,a={point:t,lineStart:function(){a.point=e},lineEnd:u,polygonStart:function(){a.lineEnd=i},polygonEnd:function(){a.lineEnd=u,a.point=t},pointRadius:function(n){return o=n,a},result:b};return a}function nr(n){function t(n){return(a?r:e)(n)}function e(t){return rr(t,function(e,r){e=n(e,r),t.point(e[0],e[1])})}function r(t){function e(e,r){e=n(e,r),t.point(e[0],e[1])}function r(){M=0/0,S.point=i,t.lineStart()}function i(e,r){var i=pe([e,r]),o=n(e,r);u(M,x,y,b,_,w,M=o[0],x=o[1],y=e,b=i[0],_=i[1],w=i[2],a,t),t.point(M,x)}function o(){S.point=e,t.lineEnd()}function c(){r(),S.point=l,S.lineEnd=s}function l(n,t){i(f=n,h=t),g=M,p=x,v=b,d=_,m=w,S.point=i}function s(){u(M,x,y,b,_,w,g,p,f,v,d,m,a,t),S.lineEnd=o,o()}var f,h,g,p,v,d,m,y,M,x,b,_,w,S={point:e,lineStart:r,lineEnd:o,polygonStart:function(){t.polygonStart(),S.lineStart=c +},polygonEnd:function(){t.polygonEnd(),S.lineStart=r}};return S}function u(t,e,r,a,c,l,s,f,h,g,p,v,d,m){var y=s-t,M=f-e,x=y*y+M*M;if(x>4*i&&d--){var b=a+g,_=c+p,w=l+v,S=Math.sqrt(b*b+_*_+w*w),k=Math.asin(w/=S),E=ga(ga(w)-1)i||ga((y*z+M*q)/x-.5)>.3||o>a*g+c*p+l*v)&&(u(t,e,r,a,c,l,N,C,E,b/=S,_/=S,w,d,m),m.point(N,C),u(N,C,E,b,_,w,s,f,h,g,p,v,d,m))}}var i=.5,o=Math.cos(30*Da),a=16;return t.precision=function(n){return arguments.length?(a=(i=n*n)>0&&16,t):Math.sqrt(i)},t}function tr(n){var t=nr(function(t,e){return n([t*Pa,e*Pa])});return function(n){return or(t(n))}}function er(n){this.stream=n}function rr(n,t){return{point:t,sphere:function(){n.sphere()},lineStart:function(){n.lineStart()},lineEnd:function(){n.lineEnd()},polygonStart:function(){n.polygonStart()},polygonEnd:function(){n.polygonEnd()}}}function ur(n){return ir(function(){return n})()}function ir(n){function t(n){return n=a(n[0]*Da,n[1]*Da),[n[0]*h+c,l-n[1]*h]}function e(n){return n=a.invert((n[0]-c)/h,(l-n[1])/h),n&&[n[0]*Pa,n[1]*Pa]}function r(){a=Ae(o=lr(m,M,x),i);var n=i(v,d);return c=g-n[0]*h,l=p+n[1]*h,u()}function u(){return s&&(s.valid=!1,s=null),t}var i,o,a,c,l,s,f=nr(function(n,t){return n=i(n,t),[n[0]*h+c,l-n[1]*h]}),h=150,g=480,p=250,v=0,d=0,m=0,M=0,x=0,b=Lc,_=y,w=null,S=null;return t.stream=function(n){return s&&(s.valid=!1),s=or(b(o,f(_(n)))),s.valid=!0,s},t.clipAngle=function(n){return arguments.length?(b=null==n?(w=n,Lc):He((w=+n)*Da),u()):w},t.clipExtent=function(n){return arguments.length?(S=n,_=n?Ie(n[0][0],n[0][1],n[1][0],n[1][1]):y,u()):S},t.scale=function(n){return arguments.length?(h=+n,r()):h},t.translate=function(n){return arguments.length?(g=+n[0],p=+n[1],r()):[g,p]},t.center=function(n){return arguments.length?(v=n[0]%360*Da,d=n[1]%360*Da,r()):[v*Pa,d*Pa]},t.rotate=function(n){return arguments.length?(m=n[0]%360*Da,M=n[1]%360*Da,x=n.length>2?n[2]%360*Da:0,r()):[m*Pa,M*Pa,x*Pa]},ta.rebind(t,f,"precision"),function(){return i=n.apply(this,arguments),t.invert=i.invert&&e,r()}}function or(n){return rr(n,function(t,e){n.point(t*Da,e*Da)})}function ar(n,t){return[n,t]}function cr(n,t){return[n>qa?n-La:-qa>n?n+La:n,t]}function lr(n,t,e){return n?t||e?Ae(fr(n),hr(t,e)):fr(n):t||e?hr(t,e):cr}function sr(n){return function(t,e){return t+=n,[t>qa?t-La:-qa>t?t+La:t,e]}}function fr(n){var t=sr(n);return t.invert=sr(-n),t}function hr(n,t){function e(n,t){var e=Math.cos(t),a=Math.cos(n)*e,c=Math.sin(n)*e,l=Math.sin(t),s=l*r+a*u;return[Math.atan2(c*i-s*o,a*r-l*u),tt(s*i+c*o)]}var r=Math.cos(n),u=Math.sin(n),i=Math.cos(t),o=Math.sin(t);return e.invert=function(n,t){var e=Math.cos(t),a=Math.cos(n)*e,c=Math.sin(n)*e,l=Math.sin(t),s=l*i-c*o;return[Math.atan2(c*i+l*o,a*r+s*u),tt(s*r-a*u)]},e}function gr(n,t){var e=Math.cos(n),r=Math.sin(n);return function(u,i,o,a){var c=o*t;null!=u?(u=pr(e,u),i=pr(e,i),(o>0?i>u:u>i)&&(u+=o*La)):(u=n+o*La,i=n-.5*c);for(var l,s=u;o>0?s>i:i>s;s-=c)a.point((l=xe([e,-r*Math.cos(s),-r*Math.sin(s)]))[0],l[1])}}function pr(n,t){var e=pe(t);e[0]-=n,Me(e);var r=nt(-e[1]);return((-e[2]<0?-r:r)+2*Math.PI-Ca)%(2*Math.PI)}function vr(n,t,e){var r=ta.range(n,t-Ca,e).concat(t);return function(n){return r.map(function(t){return[n,t]})}}function dr(n,t,e){var r=ta.range(n,t-Ca,e).concat(t);return function(n){return r.map(function(t){return[t,n]})}}function mr(n){return n.source}function yr(n){return n.target}function Mr(n,t,e,r){var u=Math.cos(t),i=Math.sin(t),o=Math.cos(r),a=Math.sin(r),c=u*Math.cos(n),l=u*Math.sin(n),s=o*Math.cos(e),f=o*Math.sin(e),h=2*Math.asin(Math.sqrt(it(r-t)+u*o*it(e-n))),g=1/Math.sin(h),p=h?function(n){var t=Math.sin(n*=h)*g,e=Math.sin(h-n)*g,r=e*c+t*s,u=e*l+t*f,o=e*i+t*a;return[Math.atan2(u,r)*Pa,Math.atan2(o,Math.sqrt(r*r+u*u))*Pa]}:function(){return[n*Pa,t*Pa]};return p.distance=h,p}function xr(){function n(n,u){var i=Math.sin(u*=Da),o=Math.cos(u),a=ga((n*=Da)-t),c=Math.cos(a);Yc+=Math.atan2(Math.sqrt((a=o*Math.sin(a))*a+(a=r*i-e*o*c)*a),e*i+r*o*c),t=n,e=i,r=o}var t,e,r;Zc.point=function(u,i){t=u*Da,e=Math.sin(i*=Da),r=Math.cos(i),Zc.point=n},Zc.lineEnd=function(){Zc.point=Zc.lineEnd=b}}function br(n,t){function e(t,e){var r=Math.cos(t),u=Math.cos(e),i=n(r*u);return[i*u*Math.sin(t),i*Math.sin(e)]}return e.invert=function(n,e){var r=Math.sqrt(n*n+e*e),u=t(r),i=Math.sin(u),o=Math.cos(u);return[Math.atan2(n*i,r*o),Math.asin(r&&e*i/r)]},e}function _r(n,t){function e(n,t){o>0?-Ra+Ca>t&&(t=-Ra+Ca):t>Ra-Ca&&(t=Ra-Ca);var e=o/Math.pow(u(t),i);return[e*Math.sin(i*n),o-e*Math.cos(i*n)]}var r=Math.cos(n),u=function(n){return Math.tan(qa/4+n/2)},i=n===t?Math.sin(n):Math.log(r/Math.cos(t))/Math.log(u(t)/u(n)),o=r*Math.pow(u(n),i)/i;return i?(e.invert=function(n,t){var e=o-t,r=K(i)*Math.sqrt(n*n+e*e);return[Math.atan2(n,e)/i,2*Math.atan(Math.pow(o/r,1/i))-Ra]},e):Sr}function wr(n,t){function e(n,t){var e=i-t;return[e*Math.sin(u*n),i-e*Math.cos(u*n)]}var r=Math.cos(n),u=n===t?Math.sin(n):(r-Math.cos(t))/(t-n),i=r/u+n;return ga(u)u;u++){for(;r>1&&Q(n[e[r-2]],n[e[r-1]],n[u])<=0;)--r;e[r++]=u}return e.slice(0,r)}function zr(n,t){return n[0]-t[0]||n[1]-t[1]}function qr(n,t,e){return(e[0]-t[0])*(n[1]-t[1])<(e[1]-t[1])*(n[0]-t[0])}function Lr(n,t,e,r){var u=n[0],i=e[0],o=t[0]-u,a=r[0]-i,c=n[1],l=e[1],s=t[1]-c,f=r[1]-l,h=(a*(c-l)-f*(u-i))/(f*o-a*s);return[u+h*o,c+h*s]}function Tr(n){var t=n[0],e=n[n.length-1];return!(t[0]-e[0]||t[1]-e[1])}function Rr(){tu(this),this.edge=this.site=this.circle=null}function Dr(n){var t=el.pop()||new Rr;return t.site=n,t}function Pr(n){Xr(n),Qc.remove(n),el.push(n),tu(n)}function Ur(n){var t=n.circle,e=t.x,r=t.cy,u={x:e,y:r},i=n.P,o=n.N,a=[n];Pr(n);for(var c=i;c.circle&&ga(e-c.circle.x)s;++s)l=a[s],c=a[s-1],Kr(l.edge,c.site,l.site,u);c=a[0],l=a[f-1],l.edge=Jr(c.site,l.site,null,u),Vr(c),Vr(l)}function jr(n){for(var t,e,r,u,i=n.x,o=n.y,a=Qc._;a;)if(r=Fr(a,o)-i,r>Ca)a=a.L;else{if(u=i-Hr(a,o),!(u>Ca)){r>-Ca?(t=a.P,e=a):u>-Ca?(t=a,e=a.N):t=e=a;break}if(!a.R){t=a;break}a=a.R}var c=Dr(n);if(Qc.insert(t,c),t||e){if(t===e)return Xr(t),e=Dr(t.site),Qc.insert(c,e),c.edge=e.edge=Jr(t.site,c.site),Vr(t),void Vr(e);if(!e)return void(c.edge=Jr(t.site,c.site));Xr(t),Xr(e);var l=t.site,s=l.x,f=l.y,h=n.x-s,g=n.y-f,p=e.site,v=p.x-s,d=p.y-f,m=2*(h*d-g*v),y=h*h+g*g,M=v*v+d*d,x={x:(d*y-g*M)/m+s,y:(h*M-v*y)/m+f};Kr(e.edge,l,p,x),c.edge=Jr(l,n,null,x),e.edge=Jr(n,p,null,x),Vr(t),Vr(e)}}function Fr(n,t){var e=n.site,r=e.x,u=e.y,i=u-t;if(!i)return r;var o=n.P;if(!o)return-1/0;e=o.site;var a=e.x,c=e.y,l=c-t;if(!l)return a;var s=a-r,f=1/i-1/l,h=s/l;return f?(-h+Math.sqrt(h*h-2*f*(s*s/(-2*l)-c+l/2+u-i/2)))/f+r:(r+a)/2}function Hr(n,t){var e=n.N;if(e)return Fr(e,t);var r=n.site;return r.y===t?r.x:1/0}function Or(n){this.site=n,this.edges=[]}function Ir(n){for(var t,e,r,u,i,o,a,c,l,s,f=n[0][0],h=n[1][0],g=n[0][1],p=n[1][1],v=Kc,d=v.length;d--;)if(i=v[d],i&&i.prepare())for(a=i.edges,c=a.length,o=0;c>o;)s=a[o].end(),r=s.x,u=s.y,l=a[++o%c].start(),t=l.x,e=l.y,(ga(r-t)>Ca||ga(u-e)>Ca)&&(a.splice(o,0,new Qr(Gr(i.site,s,ga(r-f)Ca?{x:f,y:ga(t-f)Ca?{x:ga(e-p)Ca?{x:h,y:ga(t-h)Ca?{x:ga(e-g)=-za)){var g=c*c+l*l,p=s*s+f*f,v=(f*g-l*p)/h,d=(c*p-s*g)/h,f=d+a,m=rl.pop()||new Zr;m.arc=n,m.site=u,m.x=v+o,m.y=f+Math.sqrt(v*v+d*d),m.cy=f,n.circle=m;for(var y=null,M=tl._;M;)if(m.yd||d>=a)return;if(h>p){if(i){if(i.y>=l)return}else i={x:d,y:c};e={x:d,y:l}}else{if(i){if(i.yr||r>1)if(h>p){if(i){if(i.y>=l)return}else i={x:(c-u)/r,y:c};e={x:(l-u)/r,y:l}}else{if(i){if(i.yg){if(i){if(i.x>=a)return}else i={x:o,y:r*o+u};e={x:a,y:r*a+u}}else{if(i){if(i.xi||f>o||r>h||u>g)){if(p=n.point){var p,v=t-n.x,d=e-n.y,m=v*v+d*d;if(c>m){var y=Math.sqrt(c=m);r=t-y,u=e-y,i=t+y,o=e+y,a=p}}for(var M=n.nodes,x=.5*(s+h),b=.5*(f+g),_=t>=x,w=e>=b,S=w<<1|_,k=S+4;k>S;++S)if(n=M[3&S])switch(3&S){case 0:l(n,s,f,x,b);break;case 1:l(n,x,f,h,b);break;case 2:l(n,s,b,x,g);break;case 3:l(n,x,b,h,g)}}}(n,r,u,i,o),a}function gu(n,t){n=ta.rgb(n),t=ta.rgb(t);var e=n.r,r=n.g,u=n.b,i=t.r-e,o=t.g-r,a=t.b-u;return function(n){return"#"+xt(Math.round(e+i*n))+xt(Math.round(r+o*n))+xt(Math.round(u+a*n))}}function pu(n,t){var e,r={},u={};for(e in n)e in t?r[e]=mu(n[e],t[e]):u[e]=n[e];for(e in t)e in n||(u[e]=t[e]);return function(n){for(e in r)u[e]=r[e](n);return u}}function vu(n,t){return n=+n,t=+t,function(e){return n*(1-e)+t*e}}function du(n,t){var e,r,u,i=il.lastIndex=ol.lastIndex=0,o=-1,a=[],c=[];for(n+="",t+="";(e=il.exec(n))&&(r=ol.exec(t));)(u=r.index)>i&&(u=t.slice(i,u),a[o]?a[o]+=u:a[++o]=u),(e=e[0])===(r=r[0])?a[o]?a[o]+=r:a[++o]=r:(a[++o]=null,c.push({i:o,x:vu(e,r)})),i=ol.lastIndex;return ir;++r)a[(e=c[r]).i]=e.x(n);return a.join("")})}function mu(n,t){for(var e,r=ta.interpolators.length;--r>=0&&!(e=ta.interpolators[r](n,t)););return e}function yu(n,t){var e,r=[],u=[],i=n.length,o=t.length,a=Math.min(n.length,t.length);for(e=0;a>e;++e)r.push(mu(n[e],t[e]));for(;i>e;++e)u[e]=n[e];for(;o>e;++e)u[e]=t[e];return function(n){for(e=0;a>e;++e)u[e]=r[e](n);return u}}function Mu(n){return function(t){return 0>=t?0:t>=1?1:n(t)}}function xu(n){return function(t){return 1-n(1-t)}}function bu(n){return function(t){return.5*(.5>t?n(2*t):2-n(2-2*t))}}function _u(n){return n*n}function wu(n){return n*n*n}function Su(n){if(0>=n)return 0;if(n>=1)return 1;var t=n*n,e=t*n;return 4*(.5>n?e:3*(n-t)+e-.75)}function ku(n){return function(t){return Math.pow(t,n)}}function Eu(n){return 1-Math.cos(n*Ra)}function Au(n){return Math.pow(2,10*(n-1))}function Nu(n){return 1-Math.sqrt(1-n*n)}function Cu(n,t){var e;return arguments.length<2&&(t=.45),arguments.length?e=t/La*Math.asin(1/n):(n=1,e=t/4),function(r){return 1+n*Math.pow(2,-10*r)*Math.sin((r-e)*La/t)}}function zu(n){return n||(n=1.70158),function(t){return t*t*((n+1)*t-n)}}function qu(n){return 1/2.75>n?7.5625*n*n:2/2.75>n?7.5625*(n-=1.5/2.75)*n+.75:2.5/2.75>n?7.5625*(n-=2.25/2.75)*n+.9375:7.5625*(n-=2.625/2.75)*n+.984375}function Lu(n,t){n=ta.hcl(n),t=ta.hcl(t);var e=n.h,r=n.c,u=n.l,i=t.h-e,o=t.c-r,a=t.l-u;return isNaN(o)&&(o=0,r=isNaN(r)?t.c:r),isNaN(i)?(i=0,e=isNaN(e)?t.h:e):i>180?i-=360:-180>i&&(i+=360),function(n){return st(e+i*n,r+o*n,u+a*n)+""}}function Tu(n,t){n=ta.hsl(n),t=ta.hsl(t);var e=n.h,r=n.s,u=n.l,i=t.h-e,o=t.s-r,a=t.l-u;return isNaN(o)&&(o=0,r=isNaN(r)?t.s:r),isNaN(i)?(i=0,e=isNaN(e)?t.h:e):i>180?i-=360:-180>i&&(i+=360),function(n){return ct(e+i*n,r+o*n,u+a*n)+""}}function Ru(n,t){n=ta.lab(n),t=ta.lab(t);var e=n.l,r=n.a,u=n.b,i=t.l-e,o=t.a-r,a=t.b-u;return function(n){return ht(e+i*n,r+o*n,u+a*n)+""}}function Du(n,t){return t-=n,function(e){return Math.round(n+t*e)}}function Pu(n){var t=[n.a,n.b],e=[n.c,n.d],r=ju(t),u=Uu(t,e),i=ju(Fu(e,t,-u))||0;t[0]*e[1]180?s+=360:s-l>180&&(l+=360),u.push({i:r.push(r.pop()+"rotate(",null,")")-2,x:vu(l,s)})):s&&r.push(r.pop()+"rotate("+s+")"),f!=h?u.push({i:r.push(r.pop()+"skewX(",null,")")-2,x:vu(f,h)}):h&&r.push(r.pop()+"skewX("+h+")"),g[0]!=p[0]||g[1]!=p[1]?(e=r.push(r.pop()+"scale(",null,",",null,")"),u.push({i:e-4,x:vu(g[0],p[0])},{i:e-2,x:vu(g[1],p[1])})):(1!=p[0]||1!=p[1])&&r.push(r.pop()+"scale("+p+")"),e=u.length,function(n){for(var t,i=-1;++i=0;)e.push(u[r])}function Qu(n,t){for(var e=[n],r=[];null!=(n=e.pop());)if(r.push(n),(i=n.children)&&(u=i.length))for(var u,i,o=-1;++oe;++e)(t=n[e][1])>u&&(r=e,u=t);return r}function si(n){return n.reduce(fi,0)}function fi(n,t){return n+t[1]}function hi(n,t){return gi(n,Math.ceil(Math.log(t.length)/Math.LN2+1))}function gi(n,t){for(var e=-1,r=+n[0],u=(n[1]-r)/t,i=[];++e<=t;)i[e]=u*e+r;return i}function pi(n){return[ta.min(n),ta.max(n)]}function vi(n,t){return n.value-t.value}function di(n,t){var e=n._pack_next;n._pack_next=t,t._pack_prev=n,t._pack_next=e,e._pack_prev=t}function mi(n,t){n._pack_next=t,t._pack_prev=n}function yi(n,t){var e=t.x-n.x,r=t.y-n.y,u=n.r+t.r;return.999*u*u>e*e+r*r}function Mi(n){function t(n){s=Math.min(n.x-n.r,s),f=Math.max(n.x+n.r,f),h=Math.min(n.y-n.r,h),g=Math.max(n.y+n.r,g)}if((e=n.children)&&(l=e.length)){var e,r,u,i,o,a,c,l,s=1/0,f=-1/0,h=1/0,g=-1/0;if(e.forEach(xi),r=e[0],r.x=-r.r,r.y=0,t(r),l>1&&(u=e[1],u.x=u.r,u.y=0,t(u),l>2))for(i=e[2],wi(r,u,i),t(i),di(r,i),r._pack_prev=i,di(i,u),u=r._pack_next,o=3;l>o;o++){wi(r,u,i=e[o]);var p=0,v=1,d=1;for(a=u._pack_next;a!==u;a=a._pack_next,v++)if(yi(a,i)){p=1;break}if(1==p)for(c=r._pack_prev;c!==a._pack_prev&&!yi(c,i);c=c._pack_prev,d++);p?(d>v||v==d&&u.ro;o++)i=e[o],i.x-=m,i.y-=y,M=Math.max(M,i.r+Math.sqrt(i.x*i.x+i.y*i.y));n.r=M,e.forEach(bi)}}function xi(n){n._pack_next=n._pack_prev=n}function bi(n){delete n._pack_next,delete n._pack_prev}function _i(n,t,e,r){var u=n.children;if(n.x=t+=r*n.x,n.y=e+=r*n.y,n.r*=r,u)for(var i=-1,o=u.length;++i=0;)t=u[i],t.z+=e,t.m+=e,e+=t.s+(r+=t.c)}function Ci(n,t,e){return n.a.parent===t.parent?n.a:e}function zi(n){return 1+ta.max(n,function(n){return n.y})}function qi(n){return n.reduce(function(n,t){return n+t.x},0)/n.length}function Li(n){var t=n.children;return t&&t.length?Li(t[0]):n}function Ti(n){var t,e=n.children;return e&&(t=e.length)?Ti(e[t-1]):n}function Ri(n){return{x:n.x,y:n.y,dx:n.dx,dy:n.dy}}function Di(n,t){var e=n.x+t[3],r=n.y+t[0],u=n.dx-t[1]-t[3],i=n.dy-t[0]-t[2];return 0>u&&(e+=u/2,u=0),0>i&&(r+=i/2,i=0),{x:e,y:r,dx:u,dy:i}}function Pi(n){var t=n[0],e=n[n.length-1];return e>t?[t,e]:[e,t]}function Ui(n){return n.rangeExtent?n.rangeExtent():Pi(n.range())}function ji(n,t,e,r){var u=e(n[0],n[1]),i=r(t[0],t[1]);return function(n){return i(u(n))}}function Fi(n,t){var e,r=0,u=n.length-1,i=n[r],o=n[u];return i>o&&(e=r,r=u,u=e,e=i,i=o,o=e),n[r]=t.floor(i),n[u]=t.ceil(o),n}function Hi(n){return n?{floor:function(t){return Math.floor(t/n)*n},ceil:function(t){return Math.ceil(t/n)*n}}:ml}function Oi(n,t,e,r){var u=[],i=[],o=0,a=Math.min(n.length,t.length)-1;for(n[a]2?Oi:ji,c=r?Iu:Ou;return o=u(n,t,c,e),a=u(t,n,c,mu),i}function i(n){return o(n)}var o,a;return i.invert=function(n){return a(n)},i.domain=function(t){return arguments.length?(n=t.map(Number),u()):n},i.range=function(n){return arguments.length?(t=n,u()):t},i.rangeRound=function(n){return i.range(n).interpolate(Du)},i.clamp=function(n){return arguments.length?(r=n,u()):r},i.interpolate=function(n){return arguments.length?(e=n,u()):e},i.ticks=function(t){return Xi(n,t)},i.tickFormat=function(t,e){return $i(n,t,e)},i.nice=function(t){return Zi(n,t),u()},i.copy=function(){return Ii(n,t,e,r)},u()}function Yi(n,t){return ta.rebind(n,t,"range","rangeRound","interpolate","clamp")}function Zi(n,t){return Fi(n,Hi(Vi(n,t)[2]))}function Vi(n,t){null==t&&(t=10);var e=Pi(n),r=e[1]-e[0],u=Math.pow(10,Math.floor(Math.log(r/t)/Math.LN10)),i=t/r*u;return.15>=i?u*=10:.35>=i?u*=5:.75>=i&&(u*=2),e[0]=Math.ceil(e[0]/u)*u,e[1]=Math.floor(e[1]/u)*u+.5*u,e[2]=u,e}function Xi(n,t){return ta.range.apply(ta,Vi(n,t))}function $i(n,t,e){var r=Vi(n,t);if(e){var u=ic.exec(e);if(u.shift(),"s"===u[8]){var i=ta.formatPrefix(Math.max(ga(r[0]),ga(r[1])));return u[7]||(u[7]="."+Bi(i.scale(r[2]))),u[8]="f",e=ta.format(u.join("")),function(n){return e(i.scale(n))+i.symbol}}u[7]||(u[7]="."+Wi(u[8],r)),e=u.join("")}else e=",."+Bi(r[2])+"f";return ta.format(e)}function Bi(n){return-Math.floor(Math.log(n)/Math.LN10+.01)}function Wi(n,t){var e=Bi(t[2]);return n in yl?Math.abs(e-Bi(Math.max(ga(t[0]),ga(t[1]))))+ +("e"!==n):e-2*("%"===n)}function Ji(n,t,e,r){function u(n){return(e?Math.log(0>n?0:n):-Math.log(n>0?0:-n))/Math.log(t)}function i(n){return e?Math.pow(t,n):-Math.pow(t,-n)}function o(t){return n(u(t))}return o.invert=function(t){return i(n.invert(t))},o.domain=function(t){return arguments.length?(e=t[0]>=0,n.domain((r=t.map(Number)).map(u)),o):r},o.base=function(e){return arguments.length?(t=+e,n.domain(r.map(u)),o):t},o.nice=function(){var t=Fi(r.map(u),e?Math:xl);return n.domain(t),r=t.map(i),o},o.ticks=function(){var n=Pi(r),o=[],a=n[0],c=n[1],l=Math.floor(u(a)),s=Math.ceil(u(c)),f=t%1?2:t;if(isFinite(s-l)){if(e){for(;s>l;l++)for(var h=1;f>h;h++)o.push(i(l)*h);o.push(i(l))}else for(o.push(i(l));l++0;h--)o.push(i(l)*h);for(l=0;o[l]c;s--);o=o.slice(l,s)}return o},o.tickFormat=function(n,t){if(!arguments.length)return Ml;arguments.length<2?t=Ml:"function"!=typeof t&&(t=ta.format(t));var r,a=Math.max(.1,n/o.ticks().length),c=e?(r=1e-12,Math.ceil):(r=-1e-12,Math.floor);return function(n){return n/i(c(u(n)+r))<=a?t(n):""}},o.copy=function(){return Ji(n.copy(),t,e,r)},Yi(o,n)}function Gi(n,t,e){function r(t){return n(u(t))}var u=Ki(t),i=Ki(1/t);return r.invert=function(t){return i(n.invert(t))},r.domain=function(t){return arguments.length?(n.domain((e=t.map(Number)).map(u)),r):e},r.ticks=function(n){return Xi(e,n)},r.tickFormat=function(n,t){return $i(e,n,t)},r.nice=function(n){return r.domain(Zi(e,n))},r.exponent=function(o){return arguments.length?(u=Ki(t=o),i=Ki(1/t),n.domain(e.map(u)),r):t},r.copy=function(){return Gi(n.copy(),t,e)},Yi(r,n)}function Ki(n){return function(t){return 0>t?-Math.pow(-t,n):Math.pow(t,n)}}function Qi(n,t){function e(e){return i[((u.get(e)||("range"===t.t?u.set(e,n.push(e)):0/0))-1)%i.length]}function r(t,e){return ta.range(n.length).map(function(n){return t+e*n})}var u,i,o;return e.domain=function(r){if(!arguments.length)return n;n=[],u=new l;for(var i,o=-1,a=r.length;++oe?[0/0,0/0]:[e>0?a[e-1]:n[0],et?0/0:t/i+n,[t,t+1/i]},r.copy=function(){return to(n,t,e)},u()}function eo(n,t){function e(e){return e>=e?t[ta.bisect(n,e)]:void 0}return e.domain=function(t){return arguments.length?(n=t,e):n},e.range=function(n){return arguments.length?(t=n,e):t},e.invertExtent=function(e){return e=t.indexOf(e),[n[e-1],n[e]]},e.copy=function(){return eo(n,t)},e}function ro(n){function t(n){return+n}return t.invert=t,t.domain=t.range=function(e){return arguments.length?(n=e.map(t),t):n},t.ticks=function(t){return Xi(n,t)},t.tickFormat=function(t,e){return $i(n,t,e)},t.copy=function(){return ro(n)},t}function uo(){return 0}function io(n){return n.innerRadius}function oo(n){return n.outerRadius}function ao(n){return n.startAngle}function co(n){return n.endAngle}function lo(n){return n&&n.padAngle}function so(n,t,e,r){return(n-e)*t-(t-r)*n>0?0:1}function fo(n,t,e,r,u){var i=n[0]-t[0],o=n[1]-t[1],a=(u?r:-r)/Math.sqrt(i*i+o*o),c=a*o,l=-a*i,s=n[0]+c,f=n[1]+l,h=t[0]+c,g=t[1]+l,p=(s+h)/2,v=(f+g)/2,d=h-s,m=g-f,y=d*d+m*m,M=e-r,x=s*g-h*f,b=(0>m?-1:1)*Math.sqrt(M*M*y-x*x),_=(x*m-d*b)/y,w=(-x*d-m*b)/y,S=(x*m+d*b)/y,k=(-x*d+m*b)/y,E=_-p,A=w-v,N=S-p,C=k-v;return E*E+A*A>N*N+C*C&&(_=S,w=k),[[_-c,w-l],[_*e/M,w*e/M]]}function ho(n){function t(t){function o(){l.push("M",i(n(s),a))}for(var c,l=[],s=[],f=-1,h=t.length,g=Et(e),p=Et(r);++f1&&u.push("H",r[0]),u.join("")}function mo(n){for(var t=0,e=n.length,r=n[0],u=[r[0],",",r[1]];++t1){a=t[1],i=n[c],c++,r+="C"+(u[0]+o[0])+","+(u[1]+o[1])+","+(i[0]-a[0])+","+(i[1]-a[1])+","+i[0]+","+i[1];for(var l=2;l9&&(u=3*t/Math.sqrt(u),o[a]=u*e,o[a+1]=u*r));for(a=-1;++a<=c;)u=(n[Math.min(c,a+1)][0]-n[Math.max(0,a-1)][0])/(6*(1+o[a]*o[a])),i.push([u||0,o[a]*u||0]);return i}function To(n){return n.length<3?go(n):n[0]+_o(n,Lo(n))}function Ro(n){for(var t,e,r,u=-1,i=n.length;++ur)return s();var u=i[i.active];u&&(--i.count,delete i[i.active],u.event&&u.event.interrupt.call(n,n.__data__,u.index)),i.active=r,o.event&&o.event.start.call(n,n.__data__,t),o.tween.forEach(function(e,r){(r=r.call(n,n.__data__,t))&&v.push(r)}),h=o.ease,f=o.duration,ta.timer(function(){return p.c=l(e||1)?Ne:l,1},0,a)}function l(e){if(i.active!==r)return 1;for(var u=e/f,a=h(u),c=v.length;c>0;)v[--c].call(n,a);return u>=1?(o.event&&o.event.end.call(n,n.__data__,t),s()):void 0}function s(){return--i.count?delete i[r]:delete n[e],1}var f,h,g=o.delay,p=ec,v=[];return p.t=g+a,u>=g?c(u-g):void(p.c=c)},0,a)}}function Bo(n,t,e){n.attr("transform",function(n){var r=t(n);return"translate("+(isFinite(r)?r:e(n))+",0)"})}function Wo(n,t,e){n.attr("transform",function(n){var r=t(n);return"translate(0,"+(isFinite(r)?r:e(n))+")"})}function Jo(n){return n.toISOString()}function Go(n,t,e){function r(t){return n(t)}function u(n,e){var r=n[1]-n[0],u=r/e,i=ta.bisect(Vl,u);return i==Vl.length?[t.year,Vi(n.map(function(n){return n/31536e6}),e)[2]]:i?t[u/Vl[i-1]1?{floor:function(t){for(;e(t=n.floor(t));)t=Ko(t-1);return t},ceil:function(t){for(;e(t=n.ceil(t));)t=Ko(+t+1);return t}}:n))},r.ticks=function(n,t){var e=Pi(r.domain()),i=null==n?u(e,10):"number"==typeof n?u(e,n):!n.range&&[{range:n},t];return i&&(n=i[0],t=i[1]),n.range(e[0],Ko(+e[1]+1),1>t?1:t)},r.tickFormat=function(){return e},r.copy=function(){return Go(n.copy(),t,e)},Yi(r,n)}function Ko(n){return new Date(n)}function Qo(n){return JSON.parse(n.responseText)}function na(n){var t=ua.createRange();return t.selectNode(ua.body),t.createContextualFragment(n.responseText)}var ta={version:"3.5.5"},ea=[].slice,ra=function(n){return ea.call(n)},ua=this.document;if(ua)try{ra(ua.documentElement.childNodes)[0].nodeType}catch(ia){ra=function(n){for(var t=n.length,e=new Array(t);t--;)e[t]=n[t];return e}}if(Date.now||(Date.now=function(){return+new Date}),ua)try{ua.createElement("DIV").style.setProperty("opacity",0,"")}catch(oa){var aa=this.Element.prototype,ca=aa.setAttribute,la=aa.setAttributeNS,sa=this.CSSStyleDeclaration.prototype,fa=sa.setProperty;aa.setAttribute=function(n,t){ca.call(this,n,t+"")},aa.setAttributeNS=function(n,t,e){la.call(this,n,t,e+"")},sa.setProperty=function(n,t,e){fa.call(this,n,t+"",e)}}ta.ascending=e,ta.descending=function(n,t){return n>t?-1:t>n?1:t>=n?0:0/0},ta.min=function(n,t){var e,r,u=-1,i=n.length;if(1===arguments.length){for(;++u=r){e=r;break}for(;++ur&&(e=r)}else{for(;++u=r){e=r;break}for(;++ur&&(e=r)}return e},ta.max=function(n,t){var e,r,u=-1,i=n.length;if(1===arguments.length){for(;++u=r){e=r;break}for(;++ue&&(e=r)}else{for(;++u=r){e=r;break}for(;++ue&&(e=r)}return e},ta.extent=function(n,t){var e,r,u,i=-1,o=n.length;if(1===arguments.length){for(;++i=r){e=u=r;break}for(;++ir&&(e=r),r>u&&(u=r))}else{for(;++i=r){e=u=r;break}for(;++ir&&(e=r),r>u&&(u=r))}return[e,u]},ta.sum=function(n,t){var e,r=0,i=n.length,o=-1;if(1===arguments.length)for(;++o1?c/(s-1):void 0},ta.deviation=function(){var n=ta.variance.apply(this,arguments);return n?Math.sqrt(n):n};var ha=i(e);ta.bisectLeft=ha.left,ta.bisect=ta.bisectRight=ha.right,ta.bisector=function(n){return i(1===n.length?function(t,r){return e(n(t),r)}:n)},ta.shuffle=function(n,t,e){(i=arguments.length)<3&&(e=n.length,2>i&&(t=0));for(var r,u,i=e-t;i;)u=Math.random()*i--|0,r=n[i+t],n[i+t]=n[u+t],n[u+t]=r;return n},ta.permute=function(n,t){for(var e=t.length,r=new Array(e);e--;)r[e]=n[t[e]];return r},ta.pairs=function(n){for(var t,e=0,r=n.length-1,u=n[0],i=new Array(0>r?0:r);r>e;)i[e]=[t=u,u=n[++e]];return i},ta.zip=function(){if(!(r=arguments.length))return[];for(var n=-1,t=ta.min(arguments,o),e=new Array(t);++n=0;)for(r=n[u],t=r.length;--t>=0;)e[--o]=r[t];return e};var ga=Math.abs;ta.range=function(n,t,e){if(arguments.length<3&&(e=1,arguments.length<2&&(t=n,n=0)),(t-n)/e===1/0)throw new Error("infinite range");var r,u=[],i=a(ga(e)),o=-1;if(n*=i,t*=i,e*=i,0>e)for(;(r=n+e*++o)>t;)u.push(r/i);else for(;(r=n+e*++o)=i.length)return r?r.call(u,o):e?o.sort(e):o;for(var c,s,f,h,g=-1,p=o.length,v=i[a++],d=new l;++g=i.length)return n;var r=[],u=o[e++];return n.forEach(function(n,u){r.push({key:n,values:t(u,e)})}),u?r.sort(function(n,t){return u(n.key,t.key)}):r}var e,r,u={},i=[],o=[];return u.map=function(t,e){return n(e,t,0)},u.entries=function(e){return t(n(ta.map,e,0),0)},u.key=function(n){return i.push(n),u},u.sortKeys=function(n){return o[i.length-1]=n,u},u.sortValues=function(n){return e=n,u},u.rollup=function(n){return r=n,u},u},ta.set=function(n){var t=new m;if(n)for(var e=0,r=n.length;r>e;++e)t.add(n[e]);return t},c(m,{has:h,add:function(n){return this._[s(n+="")]=!0,n},remove:g,values:p,size:v,empty:d,forEach:function(n){for(var t in this._)n.call(this,f(t))}}),ta.behavior={},ta.rebind=function(n,t){for(var e,r=1,u=arguments.length;++r=0&&(r=n.slice(e+1),n=n.slice(0,e)),n)return arguments.length<2?this[n].on(r):this[n].on(r,t);if(2===arguments.length){if(null==t)for(n in this)this.hasOwnProperty(n)&&this[n].on(r,null);return this}},ta.event=null,ta.requote=function(n){return n.replace(ma,"\\$&")};var ma=/[\\\^\$\*\+\?\|\[\]\(\)\.\{\}]/g,ya={}.__proto__?function(n,t){n.__proto__=t}:function(n,t){for(var e in t)n[e]=t[e]},Ma=function(n,t){return t.querySelector(n)},xa=function(n,t){return t.querySelectorAll(n)},ba=function(n,t){var e=n.matches||n[x(n,"matchesSelector")];return(ba=function(n,t){return e.call(n,t)})(n,t)};"function"==typeof Sizzle&&(Ma=function(n,t){return Sizzle(n,t)[0]||null},xa=Sizzle,ba=Sizzle.matchesSelector),ta.selection=function(){return ta.select(ua.documentElement)};var _a=ta.selection.prototype=[];_a.select=function(n){var t,e,r,u,i=[];n=N(n);for(var o=-1,a=this.length;++o=0&&(e=n.slice(0,t),n=n.slice(t+1)),wa.hasOwnProperty(e)?{space:wa[e],local:n}:n}},_a.attr=function(n,t){if(arguments.length<2){if("string"==typeof n){var e=this.node();return n=ta.ns.qualify(n),n.local?e.getAttributeNS(n.space,n.local):e.getAttribute(n)}for(t in n)this.each(z(t,n[t]));return this}return this.each(z(n,t))},_a.classed=function(n,t){if(arguments.length<2){if("string"==typeof n){var e=this.node(),r=(n=T(n)).length,u=-1;if(t=e.classList){for(;++uu){if("string"!=typeof n){2>u&&(e="");for(r in n)this.each(P(r,n[r],e));return this}if(2>u){var i=this.node();return t(i).getComputedStyle(i,null).getPropertyValue(n)}r=""}return this.each(P(n,e,r))},_a.property=function(n,t){if(arguments.length<2){if("string"==typeof n)return this.node()[n];for(t in n)this.each(U(t,n[t]));return this}return this.each(U(n,t))},_a.text=function(n){return arguments.length?this.each("function"==typeof n?function(){var t=n.apply(this,arguments);this.textContent=null==t?"":t}:null==n?function(){this.textContent=""}:function(){this.textContent=n}):this.node().textContent},_a.html=function(n){return arguments.length?this.each("function"==typeof n?function(){var t=n.apply(this,arguments);this.innerHTML=null==t?"":t}:null==n?function(){this.innerHTML=""}:function(){this.innerHTML=n}):this.node().innerHTML},_a.append=function(n){return n=j(n),this.select(function(){return this.appendChild(n.apply(this,arguments))})},_a.insert=function(n,t){return n=j(n),t=N(t),this.select(function(){return this.insertBefore(n.apply(this,arguments),t.apply(this,arguments)||null)})},_a.remove=function(){return this.each(F)},_a.data=function(n,t){function e(n,e){var r,u,i,o=n.length,f=e.length,h=Math.min(o,f),g=new Array(f),p=new Array(f),v=new Array(o);if(t){var d,m=new l,y=new Array(o);for(r=-1;++rr;++r)p[r]=H(e[r]);for(;o>r;++r)v[r]=n[r]}p.update=g,p.parentNode=g.parentNode=v.parentNode=n.parentNode,a.push(p),c.push(g),s.push(v)}var r,u,i=-1,o=this.length;if(!arguments.length){for(n=new Array(o=(r=this[0]).length);++ii;i++){u.push(t=[]),t.parentNode=(e=this[i]).parentNode;for(var a=0,c=e.length;c>a;a++)(r=e[a])&&n.call(r,r.__data__,a,i)&&t.push(r)}return A(u)},_a.order=function(){for(var n=-1,t=this.length;++n=0;)(e=r[u])&&(i&&i!==e.nextSibling&&i.parentNode.insertBefore(e,i),i=e);return this},_a.sort=function(n){n=I.apply(this,arguments);for(var t=-1,e=this.length;++tn;n++)for(var e=this[n],r=0,u=e.length;u>r;r++){var i=e[r];if(i)return i}return null},_a.size=function(){var n=0;return Y(this,function(){++n}),n};var Sa=[];ta.selection.enter=Z,ta.selection.enter.prototype=Sa,Sa.append=_a.append,Sa.empty=_a.empty,Sa.node=_a.node,Sa.call=_a.call,Sa.size=_a.size,Sa.select=function(n){for(var t,e,r,u,i,o=[],a=-1,c=this.length;++ar){if("string"!=typeof n){2>r&&(t=!1);for(e in n)this.each(X(e,n[e],t));return this}if(2>r)return(r=this.node()["__on"+n])&&r._;e=!1}return this.each(X(n,t,e))};var ka=ta.map({mouseenter:"mouseover",mouseleave:"mouseout"});ua&&ka.forEach(function(n){"on"+n in ua&&ka.remove(n)});var Ea,Aa=0;ta.mouse=function(n){return J(n,k())};var Na=this.navigator&&/WebKit/.test(this.navigator.userAgent)?-1:0;ta.touch=function(n,t,e){if(arguments.length<3&&(e=t,t=k().changedTouches),t)for(var r,u=0,i=t.length;i>u;++u)if((r=t[u]).identifier===e)return J(n,r)},ta.behavior.drag=function(){function n(){this.on("mousedown.drag",i).on("touchstart.drag",o)}function e(n,t,e,i,o){return function(){function a(){var n,e,r=t(h,v);r&&(n=r[0]-M[0],e=r[1]-M[1],p|=n|e,M=r,g({type:"drag",x:r[0]+l[0],y:r[1]+l[1],dx:n,dy:e}))}function c(){t(h,v)&&(m.on(i+d,null).on(o+d,null),y(p&&ta.event.target===f),g({type:"dragend"}))}var l,s=this,f=ta.event.target,h=s.parentNode,g=r.of(s,arguments),p=0,v=n(),d=".drag"+(null==v?"":"-"+v),m=ta.select(e(f)).on(i+d,a).on(o+d,c),y=W(f),M=t(h,v);u?(l=u.apply(s,arguments),l=[l.x-M[0],l.y-M[1]]):l=[0,0],g({type:"dragstart"})}}var r=E(n,"drag","dragstart","dragend"),u=null,i=e(b,ta.mouse,t,"mousemove","mouseup"),o=e(G,ta.touch,y,"touchmove","touchend");return n.origin=function(t){return arguments.length?(u=t,n):u},ta.rebind(n,r,"on")},ta.touches=function(n,t){return arguments.length<2&&(t=k().touches),t?ra(t).map(function(t){var e=J(n,t);return e.identifier=t.identifier,e}):[]};var Ca=1e-6,za=Ca*Ca,qa=Math.PI,La=2*qa,Ta=La-Ca,Ra=qa/2,Da=qa/180,Pa=180/qa,Ua=Math.SQRT2,ja=2,Fa=4;ta.interpolateZoom=function(n,t){function e(n){var t=n*y;if(m){var e=rt(v),o=i/(ja*h)*(e*ut(Ua*t+v)-et(v));return[r+o*l,u+o*s,i*e/rt(Ua*t+v)]}return[r+n*l,u+n*s,i*Math.exp(Ua*t)]}var r=n[0],u=n[1],i=n[2],o=t[0],a=t[1],c=t[2],l=o-r,s=a-u,f=l*l+s*s,h=Math.sqrt(f),g=(c*c-i*i+Fa*f)/(2*i*ja*h),p=(c*c-i*i-Fa*f)/(2*c*ja*h),v=Math.log(Math.sqrt(g*g+1)-g),d=Math.log(Math.sqrt(p*p+1)-p),m=d-v,y=(m||Math.log(c/i))/Ua;return e.duration=1e3*y,e},ta.behavior.zoom=function(){function n(n){n.on(q,f).on(Oa+".zoom",g).on("dblclick.zoom",p).on(R,h)}function e(n){return[(n[0]-k.x)/k.k,(n[1]-k.y)/k.k]}function r(n){return[n[0]*k.k+k.x,n[1]*k.k+k.y]}function u(n){k.k=Math.max(N[0],Math.min(N[1],n))}function i(n,t){t=r(t),k.x+=n[0]-t[0],k.y+=n[1]-t[1]}function o(t,e,r,o){t.__chart__={x:k.x,y:k.y,k:k.k},u(Math.pow(2,o)),i(d=e,r),t=ta.select(t),C>0&&(t=t.transition().duration(C)),t.call(n.event)}function a(){b&&b.domain(x.range().map(function(n){return(n-k.x)/k.k}).map(x.invert)),w&&w.domain(_.range().map(function(n){return(n-k.y)/k.k}).map(_.invert))}function c(n){z++||n({type:"zoomstart"})}function l(n){a(),n({type:"zoom",scale:k.k,translate:[k.x,k.y]})}function s(n){--z||n({type:"zoomend"}),d=null}function f(){function n(){f=1,i(ta.mouse(u),g),l(a)}function r(){h.on(L,null).on(T,null),p(f&&ta.event.target===o),s(a)}var u=this,o=ta.event.target,a=D.of(u,arguments),f=0,h=ta.select(t(u)).on(L,n).on(T,r),g=e(ta.mouse(u)),p=W(u);Dl.call(u),c(a)}function h(){function n(){var n=ta.touches(p);return g=k.k,n.forEach(function(n){n.identifier in d&&(d[n.identifier]=e(n))}),n}function t(){var t=ta.event.target;ta.select(t).on(x,r).on(b,a),_.push(t);for(var e=ta.event.changedTouches,u=0,i=e.length;i>u;++u)d[e[u].identifier]=null;var c=n(),l=Date.now();if(1===c.length){if(500>l-M){var s=c[0];o(p,s,d[s.identifier],Math.floor(Math.log(k.k)/Math.LN2)+1),S()}M=l}else if(c.length>1){var s=c[0],f=c[1],h=s[0]-f[0],g=s[1]-f[1];m=h*h+g*g}}function r(){var n,t,e,r,o=ta.touches(p);Dl.call(p);for(var a=0,c=o.length;c>a;++a,r=null)if(e=o[a],r=d[e.identifier]){if(t)break;n=e,t=r}if(r){var s=(s=e[0]-n[0])*s+(s=e[1]-n[1])*s,f=m&&Math.sqrt(s/m);n=[(n[0]+e[0])/2,(n[1]+e[1])/2],t=[(t[0]+r[0])/2,(t[1]+r[1])/2],u(f*g)}M=null,i(n,t),l(v)}function a(){if(ta.event.touches.length){for(var t=ta.event.changedTouches,e=0,r=t.length;r>e;++e)delete d[t[e].identifier];for(var u in d)return void n()}ta.selectAll(_).on(y,null),w.on(q,f).on(R,h),E(),s(v)}var g,p=this,v=D.of(p,arguments),d={},m=0,y=".zoom-"+ta.event.changedTouches[0].identifier,x="touchmove"+y,b="touchend"+y,_=[],w=ta.select(p),E=W(p);t(),c(v),w.on(q,null).on(R,t)}function g(){var n=D.of(this,arguments);y?clearTimeout(y):(v=e(d=m||ta.mouse(this)),Dl.call(this),c(n)),y=setTimeout(function(){y=null,s(n)},50),S(),u(Math.pow(2,.002*Ha())*k.k),i(d,v),l(n)}function p(){var n=ta.mouse(this),t=Math.log(k.k)/Math.LN2;o(this,n,e(n),ta.event.shiftKey?Math.ceil(t)-1:Math.floor(t)+1)}var v,d,m,y,M,x,b,_,w,k={x:0,y:0,k:1},A=[960,500],N=Ia,C=250,z=0,q="mousedown.zoom",L="mousemove.zoom",T="mouseup.zoom",R="touchstart.zoom",D=E(n,"zoomstart","zoom","zoomend");return Oa||(Oa="onwheel"in ua?(Ha=function(){return-ta.event.deltaY*(ta.event.deltaMode?120:1)},"wheel"):"onmousewheel"in ua?(Ha=function(){return ta.event.wheelDelta},"mousewheel"):(Ha=function(){return-ta.event.detail},"MozMousePixelScroll")),n.event=function(n){n.each(function(){var n=D.of(this,arguments),t=k;Tl?ta.select(this).transition().each("start.zoom",function(){k=this.__chart__||{x:0,y:0,k:1},c(n)}).tween("zoom:zoom",function(){var e=A[0],r=A[1],u=d?d[0]:e/2,i=d?d[1]:r/2,o=ta.interpolateZoom([(u-k.x)/k.k,(i-k.y)/k.k,e/k.k],[(u-t.x)/t.k,(i-t.y)/t.k,e/t.k]);return function(t){var r=o(t),a=e/r[2];this.__chart__=k={x:u-r[0]*a,y:i-r[1]*a,k:a},l(n)}}).each("interrupt.zoom",function(){s(n)}).each("end.zoom",function(){s(n)}):(this.__chart__=k,c(n),l(n),s(n))})},n.translate=function(t){return arguments.length?(k={x:+t[0],y:+t[1],k:k.k},a(),n):[k.x,k.y]},n.scale=function(t){return arguments.length?(k={x:k.x,y:k.y,k:+t},a(),n):k.k},n.scaleExtent=function(t){return arguments.length?(N=null==t?Ia:[+t[0],+t[1]],n):N},n.center=function(t){return arguments.length?(m=t&&[+t[0],+t[1]],n):m},n.size=function(t){return arguments.length?(A=t&&[+t[0],+t[1]],n):A},n.duration=function(t){return arguments.length?(C=+t,n):C},n.x=function(t){return arguments.length?(b=t,x=t.copy(),k={x:0,y:0,k:1},n):b},n.y=function(t){return arguments.length?(w=t,_=t.copy(),k={x:0,y:0,k:1},n):w},ta.rebind(n,D,"on")};var Ha,Oa,Ia=[0,1/0];ta.color=ot,ot.prototype.toString=function(){return this.rgb()+""},ta.hsl=at;var Ya=at.prototype=new ot;Ya.brighter=function(n){return n=Math.pow(.7,arguments.length?n:1),new at(this.h,this.s,this.l/n)},Ya.darker=function(n){return n=Math.pow(.7,arguments.length?n:1),new at(this.h,this.s,n*this.l)},Ya.rgb=function(){return ct(this.h,this.s,this.l)},ta.hcl=lt;var Za=lt.prototype=new ot;Za.brighter=function(n){return new lt(this.h,this.c,Math.min(100,this.l+Va*(arguments.length?n:1)))},Za.darker=function(n){return new lt(this.h,this.c,Math.max(0,this.l-Va*(arguments.length?n:1)))},Za.rgb=function(){return st(this.h,this.c,this.l).rgb()},ta.lab=ft;var Va=18,Xa=.95047,$a=1,Ba=1.08883,Wa=ft.prototype=new ot;Wa.brighter=function(n){return new ft(Math.min(100,this.l+Va*(arguments.length?n:1)),this.a,this.b)},Wa.darker=function(n){return new ft(Math.max(0,this.l-Va*(arguments.length?n:1)),this.a,this.b)},Wa.rgb=function(){return ht(this.l,this.a,this.b)},ta.rgb=mt;var Ja=mt.prototype=new ot;Ja.brighter=function(n){n=Math.pow(.7,arguments.length?n:1);var t=this.r,e=this.g,r=this.b,u=30;return t||e||r?(t&&u>t&&(t=u),e&&u>e&&(e=u),r&&u>r&&(r=u),new mt(Math.min(255,t/n),Math.min(255,e/n),Math.min(255,r/n))):new mt(u,u,u)},Ja.darker=function(n){return n=Math.pow(.7,arguments.length?n:1),new mt(n*this.r,n*this.g,n*this.b)},Ja.hsl=function(){return _t(this.r,this.g,this.b)},Ja.toString=function(){return"#"+xt(this.r)+xt(this.g)+xt(this.b)};var Ga=ta.map({aliceblue:15792383,antiquewhite:16444375,aqua:65535,aquamarine:8388564,azure:15794175,beige:16119260,bisque:16770244,black:0,blanchedalmond:16772045,blue:255,blueviolet:9055202,brown:10824234,burlywood:14596231,cadetblue:6266528,chartreuse:8388352,chocolate:13789470,coral:16744272,cornflowerblue:6591981,cornsilk:16775388,crimson:14423100,cyan:65535,darkblue:139,darkcyan:35723,darkgoldenrod:12092939,darkgray:11119017,darkgreen:25600,darkgrey:11119017,darkkhaki:12433259,darkmagenta:9109643,darkolivegreen:5597999,darkorange:16747520,darkorchid:10040012,darkred:9109504,darksalmon:15308410,darkseagreen:9419919,darkslateblue:4734347,darkslategray:3100495,darkslategrey:3100495,darkturquoise:52945,darkviolet:9699539,deeppink:16716947,deepskyblue:49151,dimgray:6908265,dimgrey:6908265,dodgerblue:2003199,firebrick:11674146,floralwhite:16775920,forestgreen:2263842,fuchsia:16711935,gainsboro:14474460,ghostwhite:16316671,gold:16766720,goldenrod:14329120,gray:8421504,green:32768,greenyellow:11403055,grey:8421504,honeydew:15794160,hotpink:16738740,indianred:13458524,indigo:4915330,ivory:16777200,khaki:15787660,lavender:15132410,lavenderblush:16773365,lawngreen:8190976,lemonchiffon:16775885,lightblue:11393254,lightcoral:15761536,lightcyan:14745599,lightgoldenrodyellow:16448210,lightgray:13882323,lightgreen:9498256,lightgrey:13882323,lightpink:16758465,lightsalmon:16752762,lightseagreen:2142890,lightskyblue:8900346,lightslategray:7833753,lightslategrey:7833753,lightsteelblue:11584734,lightyellow:16777184,lime:65280,limegreen:3329330,linen:16445670,magenta:16711935,maroon:8388608,mediumaquamarine:6737322,mediumblue:205,mediumorchid:12211667,mediumpurple:9662683,mediumseagreen:3978097,mediumslateblue:8087790,mediumspringgreen:64154,mediumturquoise:4772300,mediumvioletred:13047173,midnightblue:1644912,mintcream:16121850,mistyrose:16770273,moccasin:16770229,navajowhite:16768685,navy:128,oldlace:16643558,olive:8421376,olivedrab:7048739,orange:16753920,orangered:16729344,orchid:14315734,palegoldenrod:15657130,palegreen:10025880,paleturquoise:11529966,palevioletred:14381203,papayawhip:16773077,peachpuff:16767673,peru:13468991,pink:16761035,plum:14524637,powderblue:11591910,purple:8388736,rebeccapurple:6697881,red:16711680,rosybrown:12357519,royalblue:4286945,saddlebrown:9127187,salmon:16416882,sandybrown:16032864,seagreen:3050327,seashell:16774638,sienna:10506797,silver:12632256,skyblue:8900331,slateblue:6970061,slategray:7372944,slategrey:7372944,snow:16775930,springgreen:65407,steelblue:4620980,tan:13808780,teal:32896,thistle:14204888,tomato:16737095,turquoise:4251856,violet:15631086,wheat:16113331,white:16777215,whitesmoke:16119285,yellow:16776960,yellowgreen:10145074});Ga.forEach(function(n,t){Ga.set(n,yt(t))}),ta.functor=Et,ta.xhr=At(y),ta.dsv=function(n,t){function e(n,e,i){arguments.length<3&&(i=e,e=null);var o=Nt(n,t,null==e?r:u(e),i);return o.row=function(n){return arguments.length?o.response(null==(e=n)?r:u(n)):e},o}function r(n){return e.parse(n.responseText)}function u(n){return function(t){return e.parse(t.responseText,n)}}function i(t){return t.map(o).join(n)}function o(n){return a.test(n)?'"'+n.replace(/\"/g,'""')+'"':n}var a=new RegExp('["'+n+"\n]"),c=n.charCodeAt(0);return e.parse=function(n,t){var r;return e.parseRows(n,function(n,e){if(r)return r(n,e-1);var u=new Function("d","return {"+n.map(function(n,t){return JSON.stringify(n)+": d["+t+"]"}).join(",")+"}");r=t?function(n,e){return t(u(n),e)}:u})},e.parseRows=function(n,t){function e(){if(s>=l)return o;if(u)return u=!1,i;var t=s;if(34===n.charCodeAt(t)){for(var e=t;e++s;){var r=n.charCodeAt(s++),a=1;if(10===r)u=!0;else if(13===r)u=!0,10===n.charCodeAt(s)&&(++s,++a);else if(r!==c)continue;return n.slice(t,s-a)}return n.slice(t)}for(var r,u,i={},o={},a=[],l=n.length,s=0,f=0;(r=e())!==o;){for(var h=[];r!==i&&r!==o;)h.push(r),r=e();t&&null==(h=t(h,f++))||a.push(h)}return a},e.format=function(t){if(Array.isArray(t[0]))return e.formatRows(t);var r=new m,u=[];return t.forEach(function(n){for(var t in n)r.has(t)||u.push(r.add(t))}),[u.map(o).join(n)].concat(t.map(function(t){return u.map(function(n){return o(t[n])}).join(n)})).join("\n")},e.formatRows=function(n){return n.map(i).join("\n")},e},ta.csv=ta.dsv(",","text/csv"),ta.tsv=ta.dsv(" ","text/tab-separated-values");var Ka,Qa,nc,tc,ec,rc=this[x(this,"requestAnimationFrame")]||function(n){setTimeout(n,17)};ta.timer=function(n,t,e){var r=arguments.length;2>r&&(t=0),3>r&&(e=Date.now());var u=e+t,i={c:n,t:u,f:!1,n:null};Qa?Qa.n=i:Ka=i,Qa=i,nc||(tc=clearTimeout(tc),nc=1,rc(qt))},ta.timer.flush=function(){Lt(),Tt()},ta.round=function(n,t){return t?Math.round(n*(t=Math.pow(10,t)))/t:Math.round(n)};var uc=["y","z","a","f","p","n","\xb5","m","","k","M","G","T","P","E","Z","Y"].map(Dt);ta.formatPrefix=function(n,t){var e=0;return n&&(0>n&&(n*=-1),t&&(n=ta.round(n,Rt(n,t))),e=1+Math.floor(1e-12+Math.log(n)/Math.LN10),e=Math.max(-24,Math.min(24,3*Math.floor((e-1)/3)))),uc[8+e/3]};var ic=/(?:([^{])?([<>=^]))?([+\- ])?([$#])?(0)?(\d+)?(,)?(\.-?\d+)?([a-z%])?/i,oc=ta.map({b:function(n){return n.toString(2)},c:function(n){return String.fromCharCode(n)},o:function(n){return n.toString(8)},x:function(n){return n.toString(16)},X:function(n){return n.toString(16).toUpperCase()},g:function(n,t){return n.toPrecision(t)},e:function(n,t){return n.toExponential(t)},f:function(n,t){return n.toFixed(t)},r:function(n,t){return(n=ta.round(n,Rt(n,t))).toFixed(Math.max(0,Math.min(20,Rt(n*(1+1e-15),t))))}}),ac=ta.time={},cc=Date;jt.prototype={getDate:function(){return this._.getUTCDate()},getDay:function(){return this._.getUTCDay()},getFullYear:function(){return this._.getUTCFullYear()},getHours:function(){return this._.getUTCHours()},getMilliseconds:function(){return this._.getUTCMilliseconds()},getMinutes:function(){return this._.getUTCMinutes()},getMonth:function(){return this._.getUTCMonth()},getSeconds:function(){return this._.getUTCSeconds()},getTime:function(){return this._.getTime()},getTimezoneOffset:function(){return 0},valueOf:function(){return this._.valueOf()},setDate:function(){lc.setUTCDate.apply(this._,arguments)},setDay:function(){lc.setUTCDay.apply(this._,arguments)},setFullYear:function(){lc.setUTCFullYear.apply(this._,arguments)},setHours:function(){lc.setUTCHours.apply(this._,arguments)},setMilliseconds:function(){lc.setUTCMilliseconds.apply(this._,arguments)},setMinutes:function(){lc.setUTCMinutes.apply(this._,arguments)},setMonth:function(){lc.setUTCMonth.apply(this._,arguments)},setSeconds:function(){lc.setUTCSeconds.apply(this._,arguments)},setTime:function(){lc.setTime.apply(this._,arguments)}};var lc=Date.prototype;ac.year=Ft(function(n){return n=ac.day(n),n.setMonth(0,1),n},function(n,t){n.setFullYear(n.getFullYear()+t)},function(n){return n.getFullYear()}),ac.years=ac.year.range,ac.years.utc=ac.year.utc.range,ac.day=Ft(function(n){var t=new cc(2e3,0);return t.setFullYear(n.getFullYear(),n.getMonth(),n.getDate()),t},function(n,t){n.setDate(n.getDate()+t)},function(n){return n.getDate()-1}),ac.days=ac.day.range,ac.days.utc=ac.day.utc.range,ac.dayOfYear=function(n){var t=ac.year(n);return Math.floor((n-t-6e4*(n.getTimezoneOffset()-t.getTimezoneOffset()))/864e5)},["sunday","monday","tuesday","wednesday","thursday","friday","saturday"].forEach(function(n,t){t=7-t;var e=ac[n]=Ft(function(n){return(n=ac.day(n)).setDate(n.getDate()-(n.getDay()+t)%7),n},function(n,t){n.setDate(n.getDate()+7*Math.floor(t))},function(n){var e=ac.year(n).getDay();return Math.floor((ac.dayOfYear(n)+(e+t)%7)/7)-(e!==t)});ac[n+"s"]=e.range,ac[n+"s"].utc=e.utc.range,ac[n+"OfYear"]=function(n){var e=ac.year(n).getDay();return Math.floor((ac.dayOfYear(n)+(e+t)%7)/7)}}),ac.week=ac.sunday,ac.weeks=ac.sunday.range,ac.weeks.utc=ac.sunday.utc.range,ac.weekOfYear=ac.sundayOfYear;var sc={"-":"",_:" ",0:"0"},fc=/^\s*\d+/,hc=/^%/;ta.locale=function(n){return{numberFormat:Pt(n),timeFormat:Ot(n)}};var gc=ta.locale({decimal:".",thousands:",",grouping:[3],currency:["$",""],dateTime:"%a %b %e %X %Y",date:"%m/%d/%Y",time:"%H:%M:%S",periods:["AM","PM"],days:["Sunday","Monday","Tuesday","Wednesday","Thursday","Friday","Saturday"],shortDays:["Sun","Mon","Tue","Wed","Thu","Fri","Sat"],months:["January","February","March","April","May","June","July","August","September","October","November","December"],shortMonths:["Jan","Feb","Mar","Apr","May","Jun","Jul","Aug","Sep","Oct","Nov","Dec"]});ta.format=gc.numberFormat,ta.geo={},ce.prototype={s:0,t:0,add:function(n){le(n,this.t,pc),le(pc.s,this.s,this),this.s?this.t+=pc.t:this.s=pc.t +},reset:function(){this.s=this.t=0},valueOf:function(){return this.s}};var pc=new ce;ta.geo.stream=function(n,t){n&&vc.hasOwnProperty(n.type)?vc[n.type](n,t):se(n,t)};var vc={Feature:function(n,t){se(n.geometry,t)},FeatureCollection:function(n,t){for(var e=n.features,r=-1,u=e.length;++rn?4*qa+n:n,Mc.lineStart=Mc.lineEnd=Mc.point=b}};ta.geo.bounds=function(){function n(n,t){M.push(x=[s=n,h=n]),f>t&&(f=t),t>g&&(g=t)}function t(t,e){var r=pe([t*Da,e*Da]);if(m){var u=de(m,r),i=[u[1],-u[0],0],o=de(i,u);Me(o),o=xe(o);var c=t-p,l=c>0?1:-1,v=o[0]*Pa*l,d=ga(c)>180;if(d^(v>l*p&&l*t>v)){var y=o[1]*Pa;y>g&&(g=y)}else if(v=(v+360)%360-180,d^(v>l*p&&l*t>v)){var y=-o[1]*Pa;f>y&&(f=y)}else f>e&&(f=e),e>g&&(g=e);d?p>t?a(s,t)>a(s,h)&&(h=t):a(t,h)>a(s,h)&&(s=t):h>=s?(s>t&&(s=t),t>h&&(h=t)):t>p?a(s,t)>a(s,h)&&(h=t):a(t,h)>a(s,h)&&(s=t)}else n(t,e);m=r,p=t}function e(){b.point=t}function r(){x[0]=s,x[1]=h,b.point=n,m=null}function u(n,e){if(m){var r=n-p;y+=ga(r)>180?r+(r>0?360:-360):r}else v=n,d=e;Mc.point(n,e),t(n,e)}function i(){Mc.lineStart()}function o(){u(v,d),Mc.lineEnd(),ga(y)>Ca&&(s=-(h=180)),x[0]=s,x[1]=h,m=null}function a(n,t){return(t-=n)<0?t+360:t}function c(n,t){return n[0]-t[0]}function l(n,t){return t[0]<=t[1]?t[0]<=n&&n<=t[1]:nyc?(s=-(h=180),f=-(g=90)):y>Ca?g=90:-Ca>y&&(f=-90),x[0]=s,x[1]=h}};return function(n){g=h=-(s=f=1/0),M=[],ta.geo.stream(n,b);var t=M.length;if(t){M.sort(c);for(var e,r=1,u=M[0],i=[u];t>r;++r)e=M[r],l(e[0],u)||l(e[1],u)?(a(u[0],e[1])>a(u[0],u[1])&&(u[1]=e[1]),a(e[0],u[1])>a(u[0],u[1])&&(u[0]=e[0])):i.push(u=e);for(var o,e,p=-1/0,t=i.length-1,r=0,u=i[t];t>=r;u=e,++r)e=i[r],(o=a(u[1],e[0]))>p&&(p=o,s=e[0],h=u[1])}return M=x=null,1/0===s||1/0===f?[[0/0,0/0],[0/0,0/0]]:[[s,f],[h,g]]}}(),ta.geo.centroid=function(n){xc=bc=_c=wc=Sc=kc=Ec=Ac=Nc=Cc=zc=0,ta.geo.stream(n,qc);var t=Nc,e=Cc,r=zc,u=t*t+e*e+r*r;return za>u&&(t=kc,e=Ec,r=Ac,Ca>bc&&(t=_c,e=wc,r=Sc),u=t*t+e*e+r*r,za>u)?[0/0,0/0]:[Math.atan2(e,t)*Pa,tt(r/Math.sqrt(u))*Pa]};var xc,bc,_c,wc,Sc,kc,Ec,Ac,Nc,Cc,zc,qc={sphere:b,point:_e,lineStart:Se,lineEnd:ke,polygonStart:function(){qc.lineStart=Ee},polygonEnd:function(){qc.lineStart=Se}},Lc=Le(Ne,Pe,je,[-qa,-qa/2]),Tc=1e9;ta.geo.clipExtent=function(){var n,t,e,r,u,i,o={stream:function(n){return u&&(u.valid=!1),u=i(n),u.valid=!0,u},extent:function(a){return arguments.length?(i=Ie(n=+a[0][0],t=+a[0][1],e=+a[1][0],r=+a[1][1]),u&&(u.valid=!1,u=null),o):[[n,t],[e,r]]}};return o.extent([[0,0],[960,500]])},(ta.geo.conicEqualArea=function(){return Ye(Ze)}).raw=Ze,ta.geo.albers=function(){return ta.geo.conicEqualArea().rotate([96,0]).center([-.6,38.7]).parallels([29.5,45.5]).scale(1070)},ta.geo.albersUsa=function(){function n(n){var i=n[0],o=n[1];return t=null,e(i,o),t||(r(i,o),t)||u(i,o),t}var t,e,r,u,i=ta.geo.albers(),o=ta.geo.conicEqualArea().rotate([154,0]).center([-2,58.5]).parallels([55,65]),a=ta.geo.conicEqualArea().rotate([157,0]).center([-3,19.9]).parallels([8,18]),c={point:function(n,e){t=[n,e]}};return n.invert=function(n){var t=i.scale(),e=i.translate(),r=(n[0]-e[0])/t,u=(n[1]-e[1])/t;return(u>=.12&&.234>u&&r>=-.425&&-.214>r?o:u>=.166&&.234>u&&r>=-.214&&-.115>r?a:i).invert(n)},n.stream=function(n){var t=i.stream(n),e=o.stream(n),r=a.stream(n);return{point:function(n,u){t.point(n,u),e.point(n,u),r.point(n,u)},sphere:function(){t.sphere(),e.sphere(),r.sphere()},lineStart:function(){t.lineStart(),e.lineStart(),r.lineStart()},lineEnd:function(){t.lineEnd(),e.lineEnd(),r.lineEnd()},polygonStart:function(){t.polygonStart(),e.polygonStart(),r.polygonStart()},polygonEnd:function(){t.polygonEnd(),e.polygonEnd(),r.polygonEnd()}}},n.precision=function(t){return arguments.length?(i.precision(t),o.precision(t),a.precision(t),n):i.precision()},n.scale=function(t){return arguments.length?(i.scale(t),o.scale(.35*t),a.scale(t),n.translate(i.translate())):i.scale()},n.translate=function(t){if(!arguments.length)return i.translate();var l=i.scale(),s=+t[0],f=+t[1];return e=i.translate(t).clipExtent([[s-.455*l,f-.238*l],[s+.455*l,f+.238*l]]).stream(c).point,r=o.translate([s-.307*l,f+.201*l]).clipExtent([[s-.425*l+Ca,f+.12*l+Ca],[s-.214*l-Ca,f+.234*l-Ca]]).stream(c).point,u=a.translate([s-.205*l,f+.212*l]).clipExtent([[s-.214*l+Ca,f+.166*l+Ca],[s-.115*l-Ca,f+.234*l-Ca]]).stream(c).point,n},n.scale(1070)};var Rc,Dc,Pc,Uc,jc,Fc,Hc={point:b,lineStart:b,lineEnd:b,polygonStart:function(){Dc=0,Hc.lineStart=Ve},polygonEnd:function(){Hc.lineStart=Hc.lineEnd=Hc.point=b,Rc+=ga(Dc/2)}},Oc={point:Xe,lineStart:b,lineEnd:b,polygonStart:b,polygonEnd:b},Ic={point:We,lineStart:Je,lineEnd:Ge,polygonStart:function(){Ic.lineStart=Ke},polygonEnd:function(){Ic.point=We,Ic.lineStart=Je,Ic.lineEnd=Ge}};ta.geo.path=function(){function n(n){return n&&("function"==typeof a&&i.pointRadius(+a.apply(this,arguments)),o&&o.valid||(o=u(i)),ta.geo.stream(n,o)),i.result()}function t(){return o=null,n}var e,r,u,i,o,a=4.5;return n.area=function(n){return Rc=0,ta.geo.stream(n,u(Hc)),Rc},n.centroid=function(n){return _c=wc=Sc=kc=Ec=Ac=Nc=Cc=zc=0,ta.geo.stream(n,u(Ic)),zc?[Nc/zc,Cc/zc]:Ac?[kc/Ac,Ec/Ac]:Sc?[_c/Sc,wc/Sc]:[0/0,0/0]},n.bounds=function(n){return jc=Fc=-(Pc=Uc=1/0),ta.geo.stream(n,u(Oc)),[[Pc,Uc],[jc,Fc]]},n.projection=function(n){return arguments.length?(u=(e=n)?n.stream||tr(n):y,t()):e},n.context=function(n){return arguments.length?(i=null==(r=n)?new $e:new Qe(n),"function"!=typeof a&&i.pointRadius(a),t()):r},n.pointRadius=function(t){return arguments.length?(a="function"==typeof t?t:(i.pointRadius(+t),+t),n):a},n.projection(ta.geo.albersUsa()).context(null)},ta.geo.transform=function(n){return{stream:function(t){var e=new er(t);for(var r in n)e[r]=n[r];return e}}},er.prototype={point:function(n,t){this.stream.point(n,t)},sphere:function(){this.stream.sphere()},lineStart:function(){this.stream.lineStart()},lineEnd:function(){this.stream.lineEnd()},polygonStart:function(){this.stream.polygonStart()},polygonEnd:function(){this.stream.polygonEnd()}},ta.geo.projection=ur,ta.geo.projectionMutator=ir,(ta.geo.equirectangular=function(){return ur(ar)}).raw=ar.invert=ar,ta.geo.rotation=function(n){function t(t){return t=n(t[0]*Da,t[1]*Da),t[0]*=Pa,t[1]*=Pa,t}return n=lr(n[0]%360*Da,n[1]*Da,n.length>2?n[2]*Da:0),t.invert=function(t){return t=n.invert(t[0]*Da,t[1]*Da),t[0]*=Pa,t[1]*=Pa,t},t},cr.invert=ar,ta.geo.circle=function(){function n(){var n="function"==typeof r?r.apply(this,arguments):r,t=lr(-n[0]*Da,-n[1]*Da,0).invert,u=[];return e(null,null,1,{point:function(n,e){u.push(n=t(n,e)),n[0]*=Pa,n[1]*=Pa}}),{type:"Polygon",coordinates:[u]}}var t,e,r=[0,0],u=6;return n.origin=function(t){return arguments.length?(r=t,n):r},n.angle=function(r){return arguments.length?(e=gr((t=+r)*Da,u*Da),n):t},n.precision=function(r){return arguments.length?(e=gr(t*Da,(u=+r)*Da),n):u},n.angle(90)},ta.geo.distance=function(n,t){var e,r=(t[0]-n[0])*Da,u=n[1]*Da,i=t[1]*Da,o=Math.sin(r),a=Math.cos(r),c=Math.sin(u),l=Math.cos(u),s=Math.sin(i),f=Math.cos(i);return Math.atan2(Math.sqrt((e=f*o)*e+(e=l*s-c*f*a)*e),c*s+l*f*a)},ta.geo.graticule=function(){function n(){return{type:"MultiLineString",coordinates:t()}}function t(){return ta.range(Math.ceil(i/d)*d,u,d).map(h).concat(ta.range(Math.ceil(l/m)*m,c,m).map(g)).concat(ta.range(Math.ceil(r/p)*p,e,p).filter(function(n){return ga(n%d)>Ca}).map(s)).concat(ta.range(Math.ceil(a/v)*v,o,v).filter(function(n){return ga(n%m)>Ca}).map(f))}var e,r,u,i,o,a,c,l,s,f,h,g,p=10,v=p,d=90,m=360,y=2.5;return n.lines=function(){return t().map(function(n){return{type:"LineString",coordinates:n}})},n.outline=function(){return{type:"Polygon",coordinates:[h(i).concat(g(c).slice(1),h(u).reverse().slice(1),g(l).reverse().slice(1))]}},n.extent=function(t){return arguments.length?n.majorExtent(t).minorExtent(t):n.minorExtent()},n.majorExtent=function(t){return arguments.length?(i=+t[0][0],u=+t[1][0],l=+t[0][1],c=+t[1][1],i>u&&(t=i,i=u,u=t),l>c&&(t=l,l=c,c=t),n.precision(y)):[[i,l],[u,c]]},n.minorExtent=function(t){return arguments.length?(r=+t[0][0],e=+t[1][0],a=+t[0][1],o=+t[1][1],r>e&&(t=r,r=e,e=t),a>o&&(t=a,a=o,o=t),n.precision(y)):[[r,a],[e,o]]},n.step=function(t){return arguments.length?n.majorStep(t).minorStep(t):n.minorStep()},n.majorStep=function(t){return arguments.length?(d=+t[0],m=+t[1],n):[d,m]},n.minorStep=function(t){return arguments.length?(p=+t[0],v=+t[1],n):[p,v]},n.precision=function(t){return arguments.length?(y=+t,s=vr(a,o,90),f=dr(r,e,y),h=vr(l,c,90),g=dr(i,u,y),n):y},n.majorExtent([[-180,-90+Ca],[180,90-Ca]]).minorExtent([[-180,-80-Ca],[180,80+Ca]])},ta.geo.greatArc=function(){function n(){return{type:"LineString",coordinates:[t||r.apply(this,arguments),e||u.apply(this,arguments)]}}var t,e,r=mr,u=yr;return n.distance=function(){return ta.geo.distance(t||r.apply(this,arguments),e||u.apply(this,arguments))},n.source=function(e){return arguments.length?(r=e,t="function"==typeof e?null:e,n):r},n.target=function(t){return arguments.length?(u=t,e="function"==typeof t?null:t,n):u},n.precision=function(){return arguments.length?n:0},n},ta.geo.interpolate=function(n,t){return Mr(n[0]*Da,n[1]*Da,t[0]*Da,t[1]*Da)},ta.geo.length=function(n){return Yc=0,ta.geo.stream(n,Zc),Yc};var Yc,Zc={sphere:b,point:b,lineStart:xr,lineEnd:b,polygonStart:b,polygonEnd:b},Vc=br(function(n){return Math.sqrt(2/(1+n))},function(n){return 2*Math.asin(n/2)});(ta.geo.azimuthalEqualArea=function(){return ur(Vc)}).raw=Vc;var Xc=br(function(n){var t=Math.acos(n);return t&&t/Math.sin(t)},y);(ta.geo.azimuthalEquidistant=function(){return ur(Xc)}).raw=Xc,(ta.geo.conicConformal=function(){return Ye(_r)}).raw=_r,(ta.geo.conicEquidistant=function(){return Ye(wr)}).raw=wr;var $c=br(function(n){return 1/n},Math.atan);(ta.geo.gnomonic=function(){return ur($c)}).raw=$c,Sr.invert=function(n,t){return[n,2*Math.atan(Math.exp(t))-Ra]},(ta.geo.mercator=function(){return kr(Sr)}).raw=Sr;var Bc=br(function(){return 1},Math.asin);(ta.geo.orthographic=function(){return ur(Bc)}).raw=Bc;var Wc=br(function(n){return 1/(1+n)},function(n){return 2*Math.atan(n)});(ta.geo.stereographic=function(){return ur(Wc)}).raw=Wc,Er.invert=function(n,t){return[-t,2*Math.atan(Math.exp(n))-Ra]},(ta.geo.transverseMercator=function(){var n=kr(Er),t=n.center,e=n.rotate;return n.center=function(n){return n?t([-n[1],n[0]]):(n=t(),[n[1],-n[0]])},n.rotate=function(n){return n?e([n[0],n[1],n.length>2?n[2]+90:90]):(n=e(),[n[0],n[1],n[2]-90])},e([0,0,90])}).raw=Er,ta.geom={},ta.geom.hull=function(n){function t(n){if(n.length<3)return[];var t,u=Et(e),i=Et(r),o=n.length,a=[],c=[];for(t=0;o>t;t++)a.push([+u.call(this,n[t],t),+i.call(this,n[t],t),t]);for(a.sort(zr),t=0;o>t;t++)c.push([a[t][0],-a[t][1]]);var l=Cr(a),s=Cr(c),f=s[0]===l[0],h=s[s.length-1]===l[l.length-1],g=[];for(t=l.length-1;t>=0;--t)g.push(n[a[l[t]][2]]);for(t=+f;t=r&&l.x<=i&&l.y>=u&&l.y<=o?[[r,o],[i,o],[i,u],[r,u]]:[];s.point=n[a]}),t}function e(n){return n.map(function(n,t){return{x:Math.round(i(n,t)/Ca)*Ca,y:Math.round(o(n,t)/Ca)*Ca,i:t}})}var r=Ar,u=Nr,i=r,o=u,a=ul;return n?t(n):(t.links=function(n){return iu(e(n)).edges.filter(function(n){return n.l&&n.r}).map(function(t){return{source:n[t.l.i],target:n[t.r.i]}})},t.triangles=function(n){var t=[];return iu(e(n)).cells.forEach(function(e,r){for(var u,i,o=e.site,a=e.edges.sort(Yr),c=-1,l=a.length,s=a[l-1].edge,f=s.l===o?s.r:s.l;++c=l,h=r>=s,g=h<<1|f;n.leaf=!1,n=n.nodes[g]||(n.nodes[g]=su()),f?u=l:a=l,h?o=s:c=s,i(n,t,e,r,u,o,a,c)}var s,f,h,g,p,v,d,m,y,M=Et(a),x=Et(c);if(null!=t)v=t,d=e,m=r,y=u;else if(m=y=-(v=d=1/0),f=[],h=[],p=n.length,o)for(g=0;p>g;++g)s=n[g],s.xm&&(m=s.x),s.y>y&&(y=s.y),f.push(s.x),h.push(s.y);else for(g=0;p>g;++g){var b=+M(s=n[g],g),_=+x(s,g);v>b&&(v=b),d>_&&(d=_),b>m&&(m=b),_>y&&(y=_),f.push(b),h.push(_)}var w=m-v,S=y-d;w>S?y=d+w:m=v+S;var k=su();if(k.add=function(n){i(k,n,+M(n,++g),+x(n,g),v,d,m,y)},k.visit=function(n){fu(n,k,v,d,m,y)},k.find=function(n){return hu(k,n[0],n[1],v,d,m,y)},g=-1,null==t){for(;++g=0?n.slice(0,t):n,r=t>=0?n.slice(t+1):"in";return e=cl.get(e)||al,r=ll.get(r)||y,Mu(r(e.apply(null,ea.call(arguments,1))))},ta.interpolateHcl=Lu,ta.interpolateHsl=Tu,ta.interpolateLab=Ru,ta.interpolateRound=Du,ta.transform=function(n){var t=ua.createElementNS(ta.ns.prefix.svg,"g");return(ta.transform=function(n){if(null!=n){t.setAttribute("transform",n);var e=t.transform.baseVal.consolidate()}return new Pu(e?e.matrix:sl)})(n)},Pu.prototype.toString=function(){return"translate("+this.translate+")rotate("+this.rotate+")skewX("+this.skew+")scale("+this.scale+")"};var sl={a:1,b:0,c:0,d:1,e:0,f:0};ta.interpolateTransform=Hu,ta.layout={},ta.layout.bundle=function(){return function(n){for(var t=[],e=-1,r=n.length;++ea*a/d){if(p>c){var l=t.charge/c;n.px-=i*l,n.py-=o*l}return!0}if(t.point&&c&&p>c){var l=t.pointCharge/c;n.px-=i*l,n.py-=o*l}}return!t.charge}}function t(n){n.px=ta.event.x,n.py=ta.event.y,a.resume()}var e,r,u,i,o,a={},c=ta.dispatch("start","tick","end"),l=[1,1],s=.9,f=fl,h=hl,g=-30,p=gl,v=.1,d=.64,m=[],M=[];return a.tick=function(){if((r*=.99)<.005)return c.end({type:"end",alpha:r=0}),!0;var t,e,a,f,h,p,d,y,x,b=m.length,_=M.length;for(e=0;_>e;++e)a=M[e],f=a.source,h=a.target,y=h.x-f.x,x=h.y-f.y,(p=y*y+x*x)&&(p=r*i[e]*((p=Math.sqrt(p))-u[e])/p,y*=p,x*=p,h.x-=y*(d=f.weight/(h.weight+f.weight)),h.y-=x*d,f.x+=y*(d=1-d),f.y+=x*d);if((d=r*v)&&(y=l[0]/2,x=l[1]/2,e=-1,d))for(;++e0?n:0:n>0&&(c.start({type:"start",alpha:r=n}),ta.timer(a.tick)),a):r},a.start=function(){function n(n,r){if(!e){for(e=new Array(c),a=0;c>a;++a)e[a]=[];for(a=0;s>a;++a){var u=M[a];e[u.source.index].push(u.target),e[u.target.index].push(u.source)}}for(var i,o=e[t],a=-1,l=o.length;++at;++t)(r=m[t]).index=t,r.weight=0;for(t=0;s>t;++t)r=M[t],"number"==typeof r.source&&(r.source=m[r.source]),"number"==typeof r.target&&(r.target=m[r.target]),++r.source.weight,++r.target.weight;for(t=0;c>t;++t)r=m[t],isNaN(r.x)&&(r.x=n("x",p)),isNaN(r.y)&&(r.y=n("y",v)),isNaN(r.px)&&(r.px=r.x),isNaN(r.py)&&(r.py=r.y);if(u=[],"function"==typeof f)for(t=0;s>t;++t)u[t]=+f.call(this,M[t],t);else for(t=0;s>t;++t)u[t]=f;if(i=[],"function"==typeof h)for(t=0;s>t;++t)i[t]=+h.call(this,M[t],t);else for(t=0;s>t;++t)i[t]=h;if(o=[],"function"==typeof g)for(t=0;c>t;++t)o[t]=+g.call(this,m[t],t);else for(t=0;c>t;++t)o[t]=g;return a.resume()},a.resume=function(){return a.alpha(.1)},a.stop=function(){return a.alpha(0)},a.drag=function(){return e||(e=ta.behavior.drag().origin(y).on("dragstart.force",Xu).on("drag.force",t).on("dragend.force",$u)),arguments.length?void this.on("mouseover.force",Bu).on("mouseout.force",Wu).call(e):e},ta.rebind(a,c,"on")};var fl=20,hl=1,gl=1/0;ta.layout.hierarchy=function(){function n(u){var i,o=[u],a=[];for(u.depth=0;null!=(i=o.pop());)if(a.push(i),(l=e.call(n,i,i.depth))&&(c=l.length)){for(var c,l,s;--c>=0;)o.push(s=l[c]),s.parent=i,s.depth=i.depth+1;r&&(i.value=0),i.children=l}else r&&(i.value=+r.call(n,i,i.depth)||0),delete i.children;return Qu(u,function(n){var e,u;t&&(e=n.children)&&e.sort(t),r&&(u=n.parent)&&(u.value+=n.value)}),a}var t=ei,e=ni,r=ti;return n.sort=function(e){return arguments.length?(t=e,n):t},n.children=function(t){return arguments.length?(e=t,n):e},n.value=function(t){return arguments.length?(r=t,n):r},n.revalue=function(t){return r&&(Ku(t,function(n){n.children&&(n.value=0)}),Qu(t,function(t){var e;t.children||(t.value=+r.call(n,t,t.depth)||0),(e=t.parent)&&(e.value+=t.value)})),t},n},ta.layout.partition=function(){function n(t,e,r,u){var i=t.children;if(t.x=e,t.y=t.depth*u,t.dx=r,t.dy=u,i&&(o=i.length)){var o,a,c,l=-1;for(r=t.value?r/t.value:0;++lf?-1:1),p=(f-c*g)/ta.sum(l),v=ta.range(c),d=[];return null!=e&&v.sort(e===pl?function(n,t){return l[t]-l[n]}:function(n,t){return e(o[n],o[t])}),v.forEach(function(n){d[n]={data:o[n],value:a=l[n],startAngle:s,endAngle:s+=a*p+g,padAngle:h}}),d}var t=Number,e=pl,r=0,u=La,i=0;return n.value=function(e){return arguments.length?(t=e,n):t},n.sort=function(t){return arguments.length?(e=t,n):e},n.startAngle=function(t){return arguments.length?(r=t,n):r},n.endAngle=function(t){return arguments.length?(u=t,n):u},n.padAngle=function(t){return arguments.length?(i=t,n):i},n};var pl={};ta.layout.stack=function(){function n(a,c){if(!(h=a.length))return a;var l=a.map(function(e,r){return t.call(n,e,r)}),s=l.map(function(t){return t.map(function(t,e){return[i.call(n,t,e),o.call(n,t,e)]})}),f=e.call(n,s,c);l=ta.permute(l,f),s=ta.permute(s,f);var h,g,p,v,d=r.call(n,s,c),m=l[0].length;for(p=0;m>p;++p)for(u.call(n,l[0][p],v=d[p],s[0][p][1]),g=1;h>g;++g)u.call(n,l[g][p],v+=s[g-1][p][1],s[g][p][1]);return a}var t=y,e=ai,r=ci,u=oi,i=ui,o=ii;return n.values=function(e){return arguments.length?(t=e,n):t},n.order=function(t){return arguments.length?(e="function"==typeof t?t:vl.get(t)||ai,n):e},n.offset=function(t){return arguments.length?(r="function"==typeof t?t:dl.get(t)||ci,n):r},n.x=function(t){return arguments.length?(i=t,n):i},n.y=function(t){return arguments.length?(o=t,n):o},n.out=function(t){return arguments.length?(u=t,n):u},n};var vl=ta.map({"inside-out":function(n){var t,e,r=n.length,u=n.map(li),i=n.map(si),o=ta.range(r).sort(function(n,t){return u[n]-u[t]}),a=0,c=0,l=[],s=[];for(t=0;r>t;++t)e=o[t],c>a?(a+=i[e],l.push(e)):(c+=i[e],s.push(e));return s.reverse().concat(l)},reverse:function(n){return ta.range(n.length).reverse()},"default":ai}),dl=ta.map({silhouette:function(n){var t,e,r,u=n.length,i=n[0].length,o=[],a=0,c=[];for(e=0;i>e;++e){for(t=0,r=0;u>t;t++)r+=n[t][e][1];r>a&&(a=r),o.push(r)}for(e=0;i>e;++e)c[e]=(a-o[e])/2;return c},wiggle:function(n){var t,e,r,u,i,o,a,c,l,s=n.length,f=n[0],h=f.length,g=[];for(g[0]=c=l=0,e=1;h>e;++e){for(t=0,u=0;s>t;++t)u+=n[t][e][1];for(t=0,i=0,a=f[e][0]-f[e-1][0];s>t;++t){for(r=0,o=(n[t][e][1]-n[t][e-1][1])/(2*a);t>r;++r)o+=(n[r][e][1]-n[r][e-1][1])/a;i+=o*n[t][e][1]}g[e]=c-=u?i/u*a:0,l>c&&(l=c)}for(e=0;h>e;++e)g[e]-=l;return g},expand:function(n){var t,e,r,u=n.length,i=n[0].length,o=1/u,a=[];for(e=0;i>e;++e){for(t=0,r=0;u>t;t++)r+=n[t][e][1];if(r)for(t=0;u>t;t++)n[t][e][1]/=r;else for(t=0;u>t;t++)n[t][e][1]=o}for(e=0;i>e;++e)a[e]=0;return a},zero:ci});ta.layout.histogram=function(){function n(n,i){for(var o,a,c=[],l=n.map(e,this),s=r.call(this,l,i),f=u.call(this,s,l,i),i=-1,h=l.length,g=f.length-1,p=t?1:1/h;++i0)for(i=-1;++i=s[0]&&a<=s[1]&&(o=c[ta.bisect(f,a,1,g)-1],o.y+=p,o.push(n[i]));return c}var t=!0,e=Number,r=pi,u=hi;return n.value=function(t){return arguments.length?(e=t,n):e},n.range=function(t){return arguments.length?(r=Et(t),n):r},n.bins=function(t){return arguments.length?(u="number"==typeof t?function(n){return gi(n,t)}:Et(t),n):u},n.frequency=function(e){return arguments.length?(t=!!e,n):t},n},ta.layout.pack=function(){function n(n,i){var o=e.call(this,n,i),a=o[0],c=u[0],l=u[1],s=null==t?Math.sqrt:"function"==typeof t?t:function(){return t};if(a.x=a.y=0,Qu(a,function(n){n.r=+s(n.value)}),Qu(a,Mi),r){var f=r*(t?1:Math.max(2*a.r/c,2*a.r/l))/2;Qu(a,function(n){n.r+=f}),Qu(a,Mi),Qu(a,function(n){n.r-=f})}return _i(a,c/2,l/2,t?1:1/Math.max(2*a.r/c,2*a.r/l)),o}var t,e=ta.layout.hierarchy().sort(vi),r=0,u=[1,1];return n.size=function(t){return arguments.length?(u=t,n):u},n.radius=function(e){return arguments.length?(t=null==e||"function"==typeof e?e:+e,n):t},n.padding=function(t){return arguments.length?(r=+t,n):r},Gu(n,e)},ta.layout.tree=function(){function n(n,u){var s=o.call(this,n,u),f=s[0],h=t(f);if(Qu(h,e),h.parent.m=-h.z,Ku(h,r),l)Ku(f,i);else{var g=f,p=f,v=f;Ku(f,function(n){n.xp.x&&(p=n),n.depth>v.depth&&(v=n)});var d=a(g,p)/2-g.x,m=c[0]/(p.x+a(p,g)/2+d),y=c[1]/(v.depth||1);Ku(f,function(n){n.x=(n.x+d)*m,n.y=n.depth*y})}return s}function t(n){for(var t,e={A:null,children:[n]},r=[e];null!=(t=r.pop());)for(var u,i=t.children,o=0,a=i.length;a>o;++o)r.push((i[o]=u={_:i[o],parent:t,children:(u=i[o].children)&&u.slice()||[],A:null,a:null,z:0,m:0,c:0,s:0,t:null,i:o}).a=u);return e.children[0]}function e(n){var t=n.children,e=n.parent.children,r=n.i?e[n.i-1]:null;if(t.length){Ni(n);var i=(t[0].z+t[t.length-1].z)/2;r?(n.z=r.z+a(n._,r._),n.m=n.z-i):n.z=i}else r&&(n.z=r.z+a(n._,r._));n.parent.A=u(n,r,n.parent.A||e[0])}function r(n){n._.x=n.z+n.parent.m,n.m+=n.parent.m}function u(n,t,e){if(t){for(var r,u=n,i=n,o=t,c=u.parent.children[0],l=u.m,s=i.m,f=o.m,h=c.m;o=Ei(o),u=ki(u),o&&u;)c=ki(c),i=Ei(i),i.a=n,r=o.z+f-u.z-l+a(o._,u._),r>0&&(Ai(Ci(o,n,e),n,r),l+=r,s+=r),f+=o.m,l+=u.m,h+=c.m,s+=i.m;o&&!Ei(i)&&(i.t=o,i.m+=f-s),u&&!ki(c)&&(c.t=u,c.m+=l-h,e=n)}return e}function i(n){n.x*=c[0],n.y=n.depth*c[1]}var o=ta.layout.hierarchy().sort(null).value(null),a=Si,c=[1,1],l=null;return n.separation=function(t){return arguments.length?(a=t,n):a},n.size=function(t){return arguments.length?(l=null==(c=t)?i:null,n):l?null:c},n.nodeSize=function(t){return arguments.length?(l=null==(c=t)?null:i,n):l?c:null},Gu(n,o)},ta.layout.cluster=function(){function n(n,i){var o,a=t.call(this,n,i),c=a[0],l=0;Qu(c,function(n){var t=n.children;t&&t.length?(n.x=qi(t),n.y=zi(t)):(n.x=o?l+=e(n,o):0,n.y=0,o=n)});var s=Li(c),f=Ti(c),h=s.x-e(s,f)/2,g=f.x+e(f,s)/2;return Qu(c,u?function(n){n.x=(n.x-c.x)*r[0],n.y=(c.y-n.y)*r[1]}:function(n){n.x=(n.x-h)/(g-h)*r[0],n.y=(1-(c.y?n.y/c.y:1))*r[1]}),a}var t=ta.layout.hierarchy().sort(null).value(null),e=Si,r=[1,1],u=!1;return n.separation=function(t){return arguments.length?(e=t,n):e},n.size=function(t){return arguments.length?(u=null==(r=t),n):u?null:r},n.nodeSize=function(t){return arguments.length?(u=null!=(r=t),n):u?r:null},Gu(n,t)},ta.layout.treemap=function(){function n(n,t){for(var e,r,u=-1,i=n.length;++ut?0:t),e.area=isNaN(r)||0>=r?0:r}function t(e){var i=e.children;if(i&&i.length){var o,a,c,l=f(e),s=[],h=i.slice(),p=1/0,v="slice"===g?l.dx:"dice"===g?l.dy:"slice-dice"===g?1&e.depth?l.dy:l.dx:Math.min(l.dx,l.dy);for(n(h,l.dx*l.dy/e.value),s.area=0;(c=h.length)>0;)s.push(o=h[c-1]),s.area+=o.area,"squarify"!==g||(a=r(s,v))<=p?(h.pop(),p=a):(s.area-=s.pop().area,u(s,v,l,!1),v=Math.min(l.dx,l.dy),s.length=s.area=0,p=1/0);s.length&&(u(s,v,l,!0),s.length=s.area=0),i.forEach(t)}}function e(t){var r=t.children;if(r&&r.length){var i,o=f(t),a=r.slice(),c=[];for(n(a,o.dx*o.dy/t.value),c.area=0;i=a.pop();)c.push(i),c.area+=i.area,null!=i.z&&(u(c,i.z?o.dx:o.dy,o,!a.length),c.length=c.area=0);r.forEach(e)}}function r(n,t){for(var e,r=n.area,u=0,i=1/0,o=-1,a=n.length;++oe&&(i=e),e>u&&(u=e));return r*=r,t*=t,r?Math.max(t*u*p/r,r/(t*i*p)):1/0}function u(n,t,e,r){var u,i=-1,o=n.length,a=e.x,l=e.y,s=t?c(n.area/t):0;if(t==e.dx){for((r||s>e.dy)&&(s=e.dy);++ie.dx)&&(s=e.dx);++ie&&(t=1),1>e&&(n=0),function(){var e,r,u;do e=2*Math.random()-1,r=2*Math.random()-1,u=e*e+r*r;while(!u||u>1);return n+t*e*Math.sqrt(-2*Math.log(u)/u)}},logNormal:function(){var n=ta.random.normal.apply(ta,arguments);return function(){return Math.exp(n())}},bates:function(n){var t=ta.random.irwinHall(n);return function(){return t()/n}},irwinHall:function(n){return function(){for(var t=0,e=0;n>e;e++)t+=Math.random();return t}}},ta.scale={};var ml={floor:y,ceil:y};ta.scale.linear=function(){return Ii([0,1],[0,1],mu,!1)};var yl={s:1,g:1,p:1,r:1,e:1};ta.scale.log=function(){return Ji(ta.scale.linear().domain([0,1]),10,!0,[1,10])};var Ml=ta.format(".0e"),xl={floor:function(n){return-Math.ceil(-n)},ceil:function(n){return-Math.floor(-n)}};ta.scale.pow=function(){return Gi(ta.scale.linear(),1,[0,1])},ta.scale.sqrt=function(){return ta.scale.pow().exponent(.5)},ta.scale.ordinal=function(){return Qi([],{t:"range",a:[[]]})},ta.scale.category10=function(){return ta.scale.ordinal().range(bl)},ta.scale.category20=function(){return ta.scale.ordinal().range(_l)},ta.scale.category20b=function(){return ta.scale.ordinal().range(wl)},ta.scale.category20c=function(){return ta.scale.ordinal().range(Sl)};var bl=[2062260,16744206,2924588,14034728,9725885,9197131,14907330,8355711,12369186,1556175].map(Mt),_l=[2062260,11454440,16744206,16759672,2924588,10018698,14034728,16750742,9725885,12955861,9197131,12885140,14907330,16234194,8355711,13092807,12369186,14408589,1556175,10410725].map(Mt),wl=[3750777,5395619,7040719,10264286,6519097,9216594,11915115,13556636,9202993,12426809,15186514,15190932,8666169,11356490,14049643,15177372,8077683,10834324,13528509,14589654].map(Mt),Sl=[3244733,7057110,10406625,13032431,15095053,16616764,16625259,16634018,3253076,7652470,10607003,13101504,7695281,10394312,12369372,14342891,6513507,9868950,12434877,14277081].map(Mt);ta.scale.quantile=function(){return no([],[])},ta.scale.quantize=function(){return to(0,1,[0,1])},ta.scale.threshold=function(){return eo([.5],[0,1])},ta.scale.identity=function(){return ro([0,1])},ta.svg={},ta.svg.arc=function(){function n(){var n=Math.max(0,+e.apply(this,arguments)),l=Math.max(0,+r.apply(this,arguments)),s=o.apply(this,arguments)-Ra,f=a.apply(this,arguments)-Ra,h=Math.abs(f-s),g=s>f?0:1;if(n>l&&(p=l,l=n,n=p),h>=Ta)return t(l,g)+(n?t(n,1-g):"")+"Z";var p,v,d,m,y,M,x,b,_,w,S,k,E=0,A=0,N=[];if((m=(+c.apply(this,arguments)||0)/2)&&(d=i===kl?Math.sqrt(n*n+l*l):+i.apply(this,arguments),g||(A*=-1),l&&(A=tt(d/l*Math.sin(m))),n&&(E=tt(d/n*Math.sin(m)))),l){y=l*Math.cos(s+A),M=l*Math.sin(s+A),x=l*Math.cos(f-A),b=l*Math.sin(f-A);var C=Math.abs(f-s-2*A)<=qa?0:1;if(A&&so(y,M,x,b)===g^C){var z=(s+f)/2;y=l*Math.cos(z),M=l*Math.sin(z),x=b=null}}else y=M=0;if(n){_=n*Math.cos(f-E),w=n*Math.sin(f-E),S=n*Math.cos(s+E),k=n*Math.sin(s+E);var q=Math.abs(s-f+2*E)<=qa?0:1;if(E&&so(_,w,S,k)===1-g^q){var L=(s+f)/2;_=n*Math.cos(L),w=n*Math.sin(L),S=k=null}}else _=w=0;if((p=Math.min(Math.abs(l-n)/2,+u.apply(this,arguments)))>.001){v=l>n^g?0:1;var T=null==S?[_,w]:null==x?[y,M]:Lr([y,M],[S,k],[x,b],[_,w]),R=y-T[0],D=M-T[1],P=x-T[0],U=b-T[1],j=1/Math.sin(Math.acos((R*P+D*U)/(Math.sqrt(R*R+D*D)*Math.sqrt(P*P+U*U)))/2),F=Math.sqrt(T[0]*T[0]+T[1]*T[1]);if(null!=x){var H=Math.min(p,(l-F)/(j+1)),O=fo(null==S?[_,w]:[S,k],[y,M],l,H,g),I=fo([x,b],[_,w],l,H,g);p===H?N.push("M",O[0],"A",H,",",H," 0 0,",v," ",O[1],"A",l,",",l," 0 ",1-g^so(O[1][0],O[1][1],I[1][0],I[1][1]),",",g," ",I[1],"A",H,",",H," 0 0,",v," ",I[0]):N.push("M",O[0],"A",H,",",H," 0 1,",v," ",I[0])}else N.push("M",y,",",M);if(null!=S){var Y=Math.min(p,(n-F)/(j-1)),Z=fo([y,M],[S,k],n,-Y,g),V=fo([_,w],null==x?[y,M]:[x,b],n,-Y,g);p===Y?N.push("L",V[0],"A",Y,",",Y," 0 0,",v," ",V[1],"A",n,",",n," 0 ",g^so(V[1][0],V[1][1],Z[1][0],Z[1][1]),",",1-g," ",Z[1],"A",Y,",",Y," 0 0,",v," ",Z[0]):N.push("L",V[0],"A",Y,",",Y," 0 0,",v," ",Z[0])}else N.push("L",_,",",w)}else N.push("M",y,",",M),null!=x&&N.push("A",l,",",l," 0 ",C,",",g," ",x,",",b),N.push("L",_,",",w),null!=S&&N.push("A",n,",",n," 0 ",q,",",1-g," ",S,",",k);return N.push("Z"),N.join("")}function t(n,t){return"M0,"+n+"A"+n+","+n+" 0 1,"+t+" 0,"+-n+"A"+n+","+n+" 0 1,"+t+" 0,"+n}var e=io,r=oo,u=uo,i=kl,o=ao,a=co,c=lo;return n.innerRadius=function(t){return arguments.length?(e=Et(t),n):e},n.outerRadius=function(t){return arguments.length?(r=Et(t),n):r},n.cornerRadius=function(t){return arguments.length?(u=Et(t),n):u},n.padRadius=function(t){return arguments.length?(i=t==kl?kl:Et(t),n):i},n.startAngle=function(t){return arguments.length?(o=Et(t),n):o},n.endAngle=function(t){return arguments.length?(a=Et(t),n):a},n.padAngle=function(t){return arguments.length?(c=Et(t),n):c},n.centroid=function(){var n=(+e.apply(this,arguments)+ +r.apply(this,arguments))/2,t=(+o.apply(this,arguments)+ +a.apply(this,arguments))/2-Ra;return[Math.cos(t)*n,Math.sin(t)*n]},n};var kl="auto";ta.svg.line=function(){return ho(y)};var El=ta.map({linear:go,"linear-closed":po,step:vo,"step-before":mo,"step-after":yo,basis:So,"basis-open":ko,"basis-closed":Eo,bundle:Ao,cardinal:bo,"cardinal-open":Mo,"cardinal-closed":xo,monotone:To});El.forEach(function(n,t){t.key=n,t.closed=/-closed$/.test(n)});var Al=[0,2/3,1/3,0],Nl=[0,1/3,2/3,0],Cl=[0,1/6,2/3,1/6];ta.svg.line.radial=function(){var n=ho(Ro);return n.radius=n.x,delete n.x,n.angle=n.y,delete n.y,n},mo.reverse=yo,yo.reverse=mo,ta.svg.area=function(){return Do(y)},ta.svg.area.radial=function(){var n=Do(Ro);return n.radius=n.x,delete n.x,n.innerRadius=n.x0,delete n.x0,n.outerRadius=n.x1,delete n.x1,n.angle=n.y,delete n.y,n.startAngle=n.y0,delete n.y0,n.endAngle=n.y1,delete n.y1,n},ta.svg.chord=function(){function n(n,a){var c=t(this,i,n,a),l=t(this,o,n,a);return"M"+c.p0+r(c.r,c.p1,c.a1-c.a0)+(e(c,l)?u(c.r,c.p1,c.r,c.p0):u(c.r,c.p1,l.r,l.p0)+r(l.r,l.p1,l.a1-l.a0)+u(l.r,l.p1,c.r,c.p0))+"Z"}function t(n,t,e,r){var u=t.call(n,e,r),i=a.call(n,u,r),o=c.call(n,u,r)-Ra,s=l.call(n,u,r)-Ra;return{r:i,a0:o,a1:s,p0:[i*Math.cos(o),i*Math.sin(o)],p1:[i*Math.cos(s),i*Math.sin(s)]}}function e(n,t){return n.a0==t.a0&&n.a1==t.a1}function r(n,t,e){return"A"+n+","+n+" 0 "+ +(e>qa)+",1 "+t}function u(n,t,e,r){return"Q 0,0 "+r}var i=mr,o=yr,a=Po,c=ao,l=co;return n.radius=function(t){return arguments.length?(a=Et(t),n):a},n.source=function(t){return arguments.length?(i=Et(t),n):i},n.target=function(t){return arguments.length?(o=Et(t),n):o},n.startAngle=function(t){return arguments.length?(c=Et(t),n):c},n.endAngle=function(t){return arguments.length?(l=Et(t),n):l},n},ta.svg.diagonal=function(){function n(n,u){var i=t.call(this,n,u),o=e.call(this,n,u),a=(i.y+o.y)/2,c=[i,{x:i.x,y:a},{x:o.x,y:a},o];return c=c.map(r),"M"+c[0]+"C"+c[1]+" "+c[2]+" "+c[3]}var t=mr,e=yr,r=Uo;return n.source=function(e){return arguments.length?(t=Et(e),n):t},n.target=function(t){return arguments.length?(e=Et(t),n):e},n.projection=function(t){return arguments.length?(r=t,n):r},n},ta.svg.diagonal.radial=function(){var n=ta.svg.diagonal(),t=Uo,e=n.projection;return n.projection=function(n){return arguments.length?e(jo(t=n)):t},n},ta.svg.symbol=function(){function n(n,r){return(zl.get(t.call(this,n,r))||Oo)(e.call(this,n,r))}var t=Ho,e=Fo;return n.type=function(e){return arguments.length?(t=Et(e),n):t},n.size=function(t){return arguments.length?(e=Et(t),n):e},n};var zl=ta.map({circle:Oo,cross:function(n){var t=Math.sqrt(n/5)/2;return"M"+-3*t+","+-t+"H"+-t+"V"+-3*t+"H"+t+"V"+-t+"H"+3*t+"V"+t+"H"+t+"V"+3*t+"H"+-t+"V"+t+"H"+-3*t+"Z"},diamond:function(n){var t=Math.sqrt(n/(2*Ll)),e=t*Ll;return"M0,"+-t+"L"+e+",0 0,"+t+" "+-e+",0Z"},square:function(n){var t=Math.sqrt(n)/2;return"M"+-t+","+-t+"L"+t+","+-t+" "+t+","+t+" "+-t+","+t+"Z"},"triangle-down":function(n){var t=Math.sqrt(n/ql),e=t*ql/2;return"M0,"+e+"L"+t+","+-e+" "+-t+","+-e+"Z"},"triangle-up":function(n){var t=Math.sqrt(n/ql),e=t*ql/2;return"M0,"+-e+"L"+t+","+e+" "+-t+","+e+"Z"}});ta.svg.symbolTypes=zl.keys();var ql=Math.sqrt(3),Ll=Math.tan(30*Da);_a.transition=function(n){for(var t,e,r=Tl||++Ul,u=Xo(n),i=[],o=Rl||{time:Date.now(),ease:Su,delay:0,duration:250},a=-1,c=this.length;++ai;i++){u.push(t=[]);for(var e=this[i],a=0,c=e.length;c>a;a++)(r=e[a])&&n.call(r,r.__data__,a,i)&&t.push(r)}return Yo(u,this.namespace,this.id)},Pl.tween=function(n,t){var e=this.id,r=this.namespace;return arguments.length<2?this.node()[r][e].tween.get(n):Y(this,null==t?function(t){t[r][e].tween.remove(n)}:function(u){u[r][e].tween.set(n,t)})},Pl.attr=function(n,t){function e(){this.removeAttribute(a)}function r(){this.removeAttributeNS(a.space,a.local)}function u(n){return null==n?e:(n+="",function(){var t,e=this.getAttribute(a);return e!==n&&(t=o(e,n),function(n){this.setAttribute(a,t(n))})})}function i(n){return null==n?r:(n+="",function(){var t,e=this.getAttributeNS(a.space,a.local);return e!==n&&(t=o(e,n),function(n){this.setAttributeNS(a.space,a.local,t(n))})})}if(arguments.length<2){for(t in n)this.attr(t,n[t]);return this}var o="transform"==n?Hu:mu,a=ta.ns.qualify(n);return Zo(this,"attr."+n,t,a.local?i:u)},Pl.attrTween=function(n,t){function e(n,e){var r=t.call(this,n,e,this.getAttribute(u));return r&&function(n){this.setAttribute(u,r(n))}}function r(n,e){var r=t.call(this,n,e,this.getAttributeNS(u.space,u.local));return r&&function(n){this.setAttributeNS(u.space,u.local,r(n))}}var u=ta.ns.qualify(n);return this.tween("attr."+n,u.local?r:e)},Pl.style=function(n,e,r){function u(){this.style.removeProperty(n)}function i(e){return null==e?u:(e+="",function(){var u,i=t(this).getComputedStyle(this,null).getPropertyValue(n);return i!==e&&(u=mu(i,e),function(t){this.style.setProperty(n,u(t),r)})})}var o=arguments.length;if(3>o){if("string"!=typeof n){2>o&&(e="");for(r in n)this.style(r,n[r],e);return this}r=""}return Zo(this,"style."+n,e,i)},Pl.styleTween=function(n,e,r){function u(u,i){var o=e.call(this,u,i,t(this).getComputedStyle(this,null).getPropertyValue(n));return o&&function(t){this.style.setProperty(n,o(t),r)}}return arguments.length<3&&(r=""),this.tween("style."+n,u)},Pl.text=function(n){return Zo(this,"text",n,Vo)},Pl.remove=function(){var n=this.namespace;return this.each("end.transition",function(){var t;this[n].count<2&&(t=this.parentNode)&&t.removeChild(this)})},Pl.ease=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].ease:("function"!=typeof n&&(n=ta.ease.apply(ta,arguments)),Y(this,function(r){r[e][t].ease=n}))},Pl.delay=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].delay:Y(this,"function"==typeof n?function(r,u,i){r[e][t].delay=+n.call(r,r.__data__,u,i)}:(n=+n,function(r){r[e][t].delay=n}))},Pl.duration=function(n){var t=this.id,e=this.namespace;return arguments.length<1?this.node()[e][t].duration:Y(this,"function"==typeof n?function(r,u,i){r[e][t].duration=Math.max(1,n.call(r,r.__data__,u,i))}:(n=Math.max(1,n),function(r){r[e][t].duration=n}))},Pl.each=function(n,t){var e=this.id,r=this.namespace;if(arguments.length<2){var u=Rl,i=Tl;try{Tl=e,Y(this,function(t,u,i){Rl=t[r][e],n.call(t,t.__data__,u,i)})}finally{Rl=u,Tl=i}}else Y(this,function(u){var i=u[r][e];(i.event||(i.event=ta.dispatch("start","end","interrupt"))).on(n,t)});return this},Pl.transition=function(){for(var n,t,e,r,u=this.id,i=++Ul,o=this.namespace,a=[],c=0,l=this.length;l>c;c++){a.push(n=[]);for(var t=this[c],s=0,f=t.length;f>s;s++)(e=t[s])&&(r=e[o][u],$o(e,s,o,i,{time:r.time,ease:r.ease,delay:r.delay+r.duration,duration:r.duration})),n.push(e)}return Yo(a,o,i)},ta.svg.axis=function(){function n(n){n.each(function(){var n,l=ta.select(this),s=this.__chart__||e,f=this.__chart__=e.copy(),h=null==c?f.ticks?f.ticks.apply(f,a):f.domain():c,g=null==t?f.tickFormat?f.tickFormat.apply(f,a):y:t,p=l.selectAll(".tick").data(h,f),v=p.enter().insert("g",".domain").attr("class","tick").style("opacity",Ca),d=ta.transition(p.exit()).style("opacity",Ca).remove(),m=ta.transition(p.order()).style("opacity",1),M=Math.max(u,0)+o,x=Ui(f),b=l.selectAll(".domain").data([0]),_=(b.enter().append("path").attr("class","domain"),ta.transition(b));v.append("line"),v.append("text");var w,S,k,E,A=v.select("line"),N=m.select("line"),C=p.select("text").text(g),z=v.select("text"),q=m.select("text"),L="top"===r||"left"===r?-1:1;if("bottom"===r||"top"===r?(n=Bo,w="x",k="y",S="x2",E="y2",C.attr("dy",0>L?"0em":".71em").style("text-anchor","middle"),_.attr("d","M"+x[0]+","+L*i+"V0H"+x[1]+"V"+L*i)):(n=Wo,w="y",k="x",S="y2",E="x2",C.attr("dy",".32em").style("text-anchor",0>L?"end":"start"),_.attr("d","M"+L*i+","+x[0]+"H0V"+x[1]+"H"+L*i)),A.attr(E,L*u),z.attr(k,L*M),N.attr(S,0).attr(E,L*u),q.attr(w,0).attr(k,L*M),f.rangeBand){var T=f,R=T.rangeBand()/2;s=f=function(n){return T(n)+R}}else s.rangeBand?s=f:d.call(n,f,s);v.call(n,s,f),m.call(n,f,f)})}var t,e=ta.scale.linear(),r=jl,u=6,i=6,o=3,a=[10],c=null;return n.scale=function(t){return arguments.length?(e=t,n):e},n.orient=function(t){return arguments.length?(r=t in Fl?t+"":jl,n):r},n.ticks=function(){return arguments.length?(a=arguments,n):a},n.tickValues=function(t){return arguments.length?(c=t,n):c},n.tickFormat=function(e){return arguments.length?(t=e,n):t},n.tickSize=function(t){var e=arguments.length;return e?(u=+t,i=+arguments[e-1],n):u},n.innerTickSize=function(t){return arguments.length?(u=+t,n):u},n.outerTickSize=function(t){return arguments.length?(i=+t,n):i},n.tickPadding=function(t){return arguments.length?(o=+t,n):o},n.tickSubdivide=function(){return arguments.length&&n},n};var jl="bottom",Fl={top:1,right:1,bottom:1,left:1};ta.svg.brush=function(){function n(t){t.each(function(){var t=ta.select(this).style("pointer-events","all").style("-webkit-tap-highlight-color","rgba(0,0,0,0)").on("mousedown.brush",i).on("touchstart.brush",i),o=t.selectAll(".background").data([0]);o.enter().append("rect").attr("class","background").style("visibility","hidden").style("cursor","crosshair"),t.selectAll(".extent").data([0]).enter().append("rect").attr("class","extent").style("cursor","move");var a=t.selectAll(".resize").data(v,y);a.exit().remove(),a.enter().append("g").attr("class",function(n){return"resize "+n}).style("cursor",function(n){return Hl[n]}).append("rect").attr("x",function(n){return/[ew]$/.test(n)?-3:null}).attr("y",function(n){return/^[ns]/.test(n)?-3:null}).attr("width",6).attr("height",6).style("visibility","hidden"),a.style("display",n.empty()?"none":null);var c,f=ta.transition(t),h=ta.transition(o);l&&(c=Ui(l),h.attr("x",c[0]).attr("width",c[1]-c[0]),r(f)),s&&(c=Ui(s),h.attr("y",c[0]).attr("height",c[1]-c[0]),u(f)),e(f)})}function e(n){n.selectAll(".resize").attr("transform",function(n){return"translate("+f[+/e$/.test(n)]+","+h[+/^s/.test(n)]+")"})}function r(n){n.select(".extent").attr("x",f[0]),n.selectAll(".extent,.n>rect,.s>rect").attr("width",f[1]-f[0])}function u(n){n.select(".extent").attr("y",h[0]),n.selectAll(".extent,.e>rect,.w>rect").attr("height",h[1]-h[0])}function i(){function i(){32==ta.event.keyCode&&(C||(M=null,q[0]-=f[1],q[1]-=h[1],C=2),S())}function v(){32==ta.event.keyCode&&2==C&&(q[0]+=f[1],q[1]+=h[1],C=0,S())}function d(){var n=ta.mouse(b),t=!1;x&&(n[0]+=x[0],n[1]+=x[1]),C||(ta.event.altKey?(M||(M=[(f[0]+f[1])/2,(h[0]+h[1])/2]),q[0]=f[+(n[0]s?(u=r,r=s):u=s),v[0]!=r||v[1]!=u?(e?a=null:o=null,v[0]=r,v[1]=u,!0):void 0}function y(){d(),k.style("pointer-events","all").selectAll(".resize").style("display",n.empty()?"none":null),ta.select("body").style("cursor",null),L.on("mousemove.brush",null).on("mouseup.brush",null).on("touchmove.brush",null).on("touchend.brush",null).on("keydown.brush",null).on("keyup.brush",null),z(),w({type:"brushend"})}var M,x,b=this,_=ta.select(ta.event.target),w=c.of(b,arguments),k=ta.select(b),E=_.datum(),A=!/^(n|s)$/.test(E)&&l,N=!/^(e|w)$/.test(E)&&s,C=_.classed("extent"),z=W(b),q=ta.mouse(b),L=ta.select(t(b)).on("keydown.brush",i).on("keyup.brush",v);if(ta.event.changedTouches?L.on("touchmove.brush",d).on("touchend.brush",y):L.on("mousemove.brush",d).on("mouseup.brush",y),k.interrupt().selectAll("*").interrupt(),C)q[0]=f[0]-q[0],q[1]=h[0]-q[1];else if(E){var T=+/w$/.test(E),R=+/^n/.test(E);x=[f[1-T]-q[0],h[1-R]-q[1]],q[0]=f[T],q[1]=h[R]}else ta.event.altKey&&(M=q.slice());k.style("pointer-events","none").selectAll(".resize").style("display",null),ta.select("body").style("cursor",_.style("cursor")),w({type:"brushstart"}),d()}var o,a,c=E(n,"brushstart","brush","brushend"),l=null,s=null,f=[0,0],h=[0,0],g=!0,p=!0,v=Ol[0];return n.event=function(n){n.each(function(){var n=c.of(this,arguments),t={x:f,y:h,i:o,j:a},e=this.__chart__||t;this.__chart__=t,Tl?ta.select(this).transition().each("start.brush",function(){o=e.i,a=e.j,f=e.x,h=e.y,n({type:"brushstart"})}).tween("brush:brush",function(){var e=yu(f,t.x),r=yu(h,t.y);return o=a=null,function(u){f=t.x=e(u),h=t.y=r(u),n({type:"brush",mode:"resize"})}}).each("end.brush",function(){o=t.i,a=t.j,n({type:"brush",mode:"resize"}),n({type:"brushend"})}):(n({type:"brushstart"}),n({type:"brush",mode:"resize"}),n({type:"brushend"}))})},n.x=function(t){return arguments.length?(l=t,v=Ol[!l<<1|!s],n):l},n.y=function(t){return arguments.length?(s=t,v=Ol[!l<<1|!s],n):s},n.clamp=function(t){return arguments.length?(l&&s?(g=!!t[0],p=!!t[1]):l?g=!!t:s&&(p=!!t),n):l&&s?[g,p]:l?g:s?p:null},n.extent=function(t){var e,r,u,i,c;return arguments.length?(l&&(e=t[0],r=t[1],s&&(e=e[0],r=r[0]),o=[e,r],l.invert&&(e=l(e),r=l(r)),e>r&&(c=e,e=r,r=c),(e!=f[0]||r!=f[1])&&(f=[e,r])),s&&(u=t[0],i=t[1],l&&(u=u[1],i=i[1]),a=[u,i],s.invert&&(u=s(u),i=s(i)),u>i&&(c=u,u=i,i=c),(u!=h[0]||i!=h[1])&&(h=[u,i])),n):(l&&(o?(e=o[0],r=o[1]):(e=f[0],r=f[1],l.invert&&(e=l.invert(e),r=l.invert(r)),e>r&&(c=e,e=r,r=c))),s&&(a?(u=a[0],i=a[1]):(u=h[0],i=h[1],s.invert&&(u=s.invert(u),i=s.invert(i)),u>i&&(c=u,u=i,i=c))),l&&s?[[e,u],[r,i]]:l?[e,r]:s&&[u,i])},n.clear=function(){return n.empty()||(f=[0,0],h=[0,0],o=a=null),n},n.empty=function(){return!!l&&f[0]==f[1]||!!s&&h[0]==h[1]},ta.rebind(n,c,"on")};var Hl={n:"ns-resize",e:"ew-resize",s:"ns-resize",w:"ew-resize",nw:"nwse-resize",ne:"nesw-resize",se:"nwse-resize",sw:"nesw-resize"},Ol=[["n","e","s","w","nw","ne","se","sw"],["e","w"],["n","s"],[]],Il=ac.format=gc.timeFormat,Yl=Il.utc,Zl=Yl("%Y-%m-%dT%H:%M:%S.%LZ");Il.iso=Date.prototype.toISOString&&+new Date("2000-01-01T00:00:00.000Z")?Jo:Zl,Jo.parse=function(n){var t=new Date(n);return isNaN(t)?null:t},Jo.toString=Zl.toString,ac.second=Ft(function(n){return new cc(1e3*Math.floor(n/1e3))},function(n,t){n.setTime(n.getTime()+1e3*Math.floor(t))},function(n){return n.getSeconds()}),ac.seconds=ac.second.range,ac.seconds.utc=ac.second.utc.range,ac.minute=Ft(function(n){return new cc(6e4*Math.floor(n/6e4))},function(n,t){n.setTime(n.getTime()+6e4*Math.floor(t))},function(n){return n.getMinutes()}),ac.minutes=ac.minute.range,ac.minutes.utc=ac.minute.utc.range,ac.hour=Ft(function(n){var t=n.getTimezoneOffset()/60;return new cc(36e5*(Math.floor(n/36e5-t)+t))},function(n,t){n.setTime(n.getTime()+36e5*Math.floor(t))},function(n){return n.getHours()}),ac.hours=ac.hour.range,ac.hours.utc=ac.hour.utc.range,ac.month=Ft(function(n){return n=ac.day(n),n.setDate(1),n},function(n,t){n.setMonth(n.getMonth()+t)},function(n){return n.getMonth()}),ac.months=ac.month.range,ac.months.utc=ac.month.utc.range;var Vl=[1e3,5e3,15e3,3e4,6e4,3e5,9e5,18e5,36e5,108e5,216e5,432e5,864e5,1728e5,6048e5,2592e6,7776e6,31536e6],Xl=[[ac.second,1],[ac.second,5],[ac.second,15],[ac.second,30],[ac.minute,1],[ac.minute,5],[ac.minute,15],[ac.minute,30],[ac.hour,1],[ac.hour,3],[ac.hour,6],[ac.hour,12],[ac.day,1],[ac.day,2],[ac.week,1],[ac.month,1],[ac.month,3],[ac.year,1]],$l=Il.multi([[".%L",function(n){return n.getMilliseconds()}],[":%S",function(n){return n.getSeconds()}],["%I:%M",function(n){return n.getMinutes()}],["%I %p",function(n){return n.getHours()}],["%a %d",function(n){return n.getDay()&&1!=n.getDate()}],["%b %d",function(n){return 1!=n.getDate()}],["%B",function(n){return n.getMonth()}],["%Y",Ne]]),Bl={range:function(n,t,e){return ta.range(Math.ceil(n/e)*e,+t,e).map(Ko)},floor:y,ceil:y};Xl.year=ac.year,ac.scale=function(){return Go(ta.scale.linear(),Xl,$l)};var Wl=Xl.map(function(n){return[n[0].utc,n[1]]}),Jl=Yl.multi([[".%L",function(n){return n.getUTCMilliseconds()}],[":%S",function(n){return n.getUTCSeconds()}],["%I:%M",function(n){return n.getUTCMinutes()}],["%I %p",function(n){return n.getUTCHours()}],["%a %d",function(n){return n.getUTCDay()&&1!=n.getUTCDate()}],["%b %d",function(n){return 1!=n.getUTCDate()}],["%B",function(n){return n.getUTCMonth()}],["%Y",Ne]]);Wl.year=ac.year.utc,ac.scale.utc=function(){return Go(ta.scale.linear(),Wl,Jl)},ta.text=At(function(n){return n.responseText}),ta.json=function(n,t){return Nt(n,"application/json",Qo,t)},ta.html=function(n,t){return Nt(n,"text/html",na,t)},ta.xml=At(function(n){return n.responseXML}),"function"==typeof define&&define.amd?define(ta):"object"==typeof module&&module.exports&&(module.exports=ta),this.d3=ta}(); diff --git a/dpark/web/ui/static/js/dag-viz.js b/dpark/web/ui/static/js/dag-viz.js new file mode 100644 index 00000000..2453eb41 --- /dev/null +++ b/dpark/web/ui/static/js/dag-viz.js @@ -0,0 +1,517 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +/* + * This file contains the logic to render the RDD DAG visualization in the UI. + * + * This DAG describes the relationships between + * (1) an RDD and its dependencies, + * (2) an RDD and its operation scopes, and + * (3) an RDD's operation scopes and the stage / job hierarchy + * + * An operation scope is a general, named code block that instantiates RDDs + * (e.g. filter, textFile, reduceByKey). An operation scope can be nested inside + * of other scopes if the corresponding RDD operation invokes other such operations + * (for more detail, see o.a.s.rdd.RDDOperationScope). + * + * A stage may include one or more operation scopes if the RDD operations are + * streamlined into one stage (e.g. rdd.map(...).filter(...).flatMap(...)). + * On the flip side, an operation scope may also include one or many stages, + * or even jobs if the RDD operation is higher level than Spark's scheduling + * primitives (e.g. take, any SQL query). + * + * In the visualization, an RDD is expressed as a node, and its dependencies + * as directed edges (from parent to child). operation scopes, stages, and + * jobs are expressed as clusters that may contain one or many nodes. These + * clusters may be nested inside of each other in the scenarios described + * above. + * + * The visualization is rendered in an SVG contained in "div#dag-viz-graph", + * and its input data is expected to be populated in "div#dag-viz-metadata" + * by Spark's UI code. This is currently used only on the stage page and on + * the job page. + * + * This requires jQuery, d3, and dagre-d3. Note that we use a custom release + * of dagre-d3 (http://github.com/andrewor14/dagre-d3) for some specific + * functionality. For more detail, please track the changes in that project + * since it was forked (commit 101503833a8ce5fe369547f6addf3e71172ce10b). + */ + +var VizConstants = { + svgMarginX: 16, + svgMarginY: 16, + stageSep: 40, + graphPrefix: "graph_", + nodePrefix: "node_", + clusterPrefix: "cluster_" +}; + +var JobPageVizConstants = { + clusterLabelSize: 12, + stageClusterLabelSize: 14, + rankSep: 40 +}; + +var StagePageVizConstants = { + clusterLabelSize: 14, + stageClusterLabelSize: 14, + rankSep: 40 +}; + +/* + * Return "expand-dag-viz-arrow-job" if forJob is true. + * Otherwise, return "expand-dag-viz-arrow-stage". + */ +function expandDagVizArrowKey(forJob) { + return forJob ? "expand-dag-viz-arrow-job" : "expand-dag-viz-arrow-stage"; +} + +/* + * Show or hide the RDD DAG visualization. + * + * The graph is only rendered the first time this is called. + * This is the narrow interface called from the Scala UI code. + */ +function toggleDagViz(forJob) { + status = !status; + + var arrowSelector = ".expand-dag-viz-arrow"; + $(arrowSelector).toggleClass('arrow-closed'); + $(arrowSelector).toggleClass('arrow-open'); + var shouldShow = $(arrowSelector).hasClass("arrow-open"); + if (shouldShow) { + var shouldRender = graphContainer().select("*").empty(); + if (shouldRender) { + renderDagViz(forJob); + } + graphContainer().style("display", "block"); + } else { + // Save the graph for later so we don't have to render it again + graphContainer().style("display", "none"); + } + + window.localStorage.setItem(expandDagVizArrowKey(forJob), "" + status); +} + +$(function (){ + window.localStorage.setItem(expandDagVizArrowKey(true), "true") + if ($("#stage-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(false)) == "true") { + // Set it to false so that the click function can revert it + window.localStorage.setItem(expandDagVizArrowKey(false), "false"); + toggleDagViz(false); + } else if ($("#job-dag-viz").length && + window.localStorage.getItem(expandDagVizArrowKey(true)) == "true") { + // Set it to false so that the click function can revert it + window.localStorage.setItem(expandDagVizArrowKey(true), "false"); + toggleDagViz(true); + } +}); + +/* + * Render the RDD DAG visualization. + * + * Input DOM hierarchy: + * div#dag-viz-metadata > + * div.stage-metadata > + * div.[dot-file | incoming-edge | outgoing-edge] + * + * Output DOM hierarchy: + * div#dag-viz-graph > + * svg > + * g.cluster_stage_[stageId] + * + * Note that the input metadata is populated by o.a.s.ui.UIUtils.showDagViz. + * Any changes in the input format here must be reflected there. + */ +function renderDagViz(forJob) { + + // If there is not a dot file to render, fail fast and report error + var jobOrStage = forJob ? "job" : "stage"; + if (metadataContainer().empty() || + metadataContainer().selectAll("div").empty()) { + var message = + "No visualization information available for this " + jobOrStage + "!
" + + "If this is an old " + jobOrStage + ", its visualization metadata may have been " + + "cleaned up over time.
You may consider increasing the value of "; + if (forJob) { + message += "spark.ui.retainedJobs and spark.ui.retainedStages."; + } else { + message += "spark.ui.retainedStages"; + } + graphContainer().append("div").attr("id", "empty-dag-viz-message").html(message); + return; + } + + // Render + var svg = graphContainer().append("svg").attr("class", jobOrStage); + if (forJob) { + renderDagVizForJob(svg); + } else { + renderDagVizForStage(svg); + } + + // Find cached RDDs and mark them as such + metadataContainer().selectAll(".cached-rdd").each(function(v) { + var rddId = d3.select(this).text().trim(); + var nodeId = VizConstants.nodePrefix + rddId; + svg.selectAll("g." + nodeId).classed("cached", true); + }); + + resizeSvg(svg); +} + +/* Render the RDD DAG visualization on the stage page. */ +function renderDagVizForStage(svgContainer) { + var metadata = metadataContainer().select(".stage-metadata"); + var dot = metadata.select(".dot-file").text().trim(); + var containerId = VizConstants.graphPrefix + metadata.attr("stage-id"); + var container = svgContainer.append("g").attr("id", containerId); + renderDot(dot, container, false); + + // Round corners on rectangles + svgContainer + .selectAll("rect") + .attr("rx", "5") + .attr("ry", "5"); +} + +/* + * Render the RDD DAG visualization on the job page. + * + * Due to limitations in dagre-d3, each stage is rendered independently so that + * we have more control on how to position them. Unfortunately, this means we + * cannot rely on dagre-d3 to render edges that cross stages and must render + * these manually on our own. + */ +function renderDagVizForJob(svgContainer) { + var crossStageEdges = []; + + // Each div.stage-metadata contains the information needed to generate the graph + // for a stage. This includes the DOT file produced from the appropriate UI listener, + // any incoming and outgoing edges, and any cached RDDs that belong to this stage. + metadataContainer().selectAll(".stage-metadata").each(function(d, i) { + var metadata = d3.select(this); + var dot = metadata.select(".dot-file").text(); + var stageId = metadata.attr("stage-id"); + var containerId = VizConstants.graphPrefix + stageId; + var isSkipped = metadata.attr("skipped") == "true"; + var container; + if (isSkipped) { + container = svgContainer + .append("g") + .attr("id", containerId) + .attr("skipped", "true"); + } else { + // Link each graph to the corresponding stage page (TODO: handle stage attempts) + // Use the link from the stage table so it also works for the history server + var attemptId = 0 + container = svgContainer + .append("a") + .attr("onclick", "window.localStorage.setItem(expandDagVizArrowKey(false), true)") + .append("g") + .attr("id", containerId); + } + + // Now we need to shift the container for this stage so it doesn't overlap with + // existing ones, taking into account the position and width of the last stage's + // container. We do not need to do this for the first stage of this job. + if (i > 0) { + var existingStages = svgContainer.selectAll("g.cluster.stage") + if (!existingStages.empty()) { + var lastStage = d3.select(existingStages[0].pop()); + var lastStageWidth = toFloat(lastStage.select("rect").attr("width")); + var lastStagePosition = getAbsolutePosition(lastStage); + var offset = lastStagePosition.x + lastStageWidth + VizConstants.stageSep; + container.attr("transform", "translate(" + offset + ", 0)"); + } + } + + // Actually render the stage + renderDot(dot, container, true); + + // Mark elements as skipped if appropriate. Unfortunately we need to mark all + // elements instead of the parent container because of CSS override rules. + if (isSkipped) { + container.selectAll("g").classed("skipped", true); + } + + // Round corners on rectangles + container + .selectAll("rect") + .attr("rx", "4") + .attr("ry", "4"); + + // If there are any incoming edges into this graph, keep track of them to render + // them separately later. Note that we cannot draw them now because we need to + // put these edges in a separate container that is on top of all stage graphs. + metadata.selectAll(".incoming-edge").each(function(v) { + var edge = d3.select(this).text().trim().split(","); // e.g. 3,4 => [3, 4] + crossStageEdges.push(edge); + }); + }); + + addTooltipsForRDDs(svgContainer); + drawCrossStageEdges(crossStageEdges, svgContainer); +} + +/* Render the dot file as an SVG in the given container. */ +function renderDot(dot, container, forJob) { + var escaped_dot = dot + .replace(/</g, "<") + .replace(/>/g, ">") + .replace(/"/g, "\""); + var g = graphlibDot.read(escaped_dot); + var renderer = new dagreD3.render(); + preprocessGraphLayout(g, forJob); + renderer(container, g); + + // Find the stage cluster and mark it for styling and post-processing + container.selectAll("g.cluster[name^=\"Stage \"]").classed("stage", true); +} + +/* -------------------- * + * | Helper functions | * + * -------------------- */ + +// Helper d3 accessors +function graphContainer() { return d3.select("#dag-viz-graph"); } +function metadataContainer() { return d3.select("#dag-viz-metadata"); } + +/* + * Helper function to pre-process the graph layout. + * This step is necessary for certain styles that affect the positioning + * and sizes of graph elements, e.g. padding, font style, shape. + */ +function preprocessGraphLayout(g, forJob) { + var nodes = g.nodes(); + for (var i = 0; i < nodes.length; i++) { + var isCluster = g.children(nodes[i]).length > 0; + if (!isCluster) { + var node = g.node(nodes[i]); + if (forJob) { + // Do not display RDD name on job page + node.shape = "circle"; + node.labelStyle = "font-size: 0px"; + } else { + node.labelStyle = "font-size: 12px"; + } + node.padding = "5"; + } + } + // Curve the edges + var edges = g.edges(); + for (var j = 0; j < edges.length; j++) { + var edge = g.edge(edges[j]); + edge.lineInterpolate = "basis"; + } + // Adjust vertical separation between nodes + if (forJob) { + g.graph().rankSep = JobPageVizConstants.rankSep; + } else { + g.graph().rankSep = StagePageVizConstants.rankSep; + } +} + +/* + * Helper function to size the SVG appropriately such that all elements are displyed. + * This assumes that all outermost elements are clusters (rectangles). + */ +function resizeSvg(svg) { + var allClusters = svg.selectAll("g.cluster rect")[0]; + var startX = -VizConstants.svgMarginX + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).x; + })); + var startY = -VizConstants.svgMarginY + + toFloat(d3.min(allClusters, function(e) { + return getAbsolutePosition(d3.select(e)).y; + })); + var endX = VizConstants.svgMarginX + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e); + return getAbsolutePosition(t).x + toFloat(t.attr("width")); + })); + var endY = VizConstants.svgMarginY + + toFloat(d3.max(allClusters, function(e) { + var t = d3.select(e); + return getAbsolutePosition(t).y + toFloat(t.attr("height")); + })); + var width = endX - startX; + var height = endY - startY; + svg.attr("viewBox", startX + " " + startY + " " + width + " " + height) + .attr("width", width) + .attr("height", height); +} + +/* + * (Job page only) Helper function to draw edges that cross stage boundaries. + * We need to do this manually because we render each stage separately in dagre-d3. + */ +function drawCrossStageEdges(edges, svgContainer) { + if (edges.length == 0) { + return; + } + // Draw the paths first + var edgesContainer = svgContainer.append("g").attr("id", "cross-stage-edges"); + for (var i = 0; i < edges.length; i++) { + var fromRDDId = edges[i][0]; + var toRDDId = edges[i][1]; + connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer); + } + // Now draw the arrows by borrowing the arrow marker generated by dagre-d3 + var dagreD3Marker = svgContainer.select("g.edgePaths marker"); + if (!dagreD3Marker.empty()) { + svgContainer + .append(function() { return dagreD3Marker.node().cloneNode(true); }) + .attr("id", "marker-arrow"); + svgContainer.selectAll("g > path").attr("marker-end", "url(#marker-arrow)"); + svgContainer.selectAll("g.edgePaths def").remove(); // We no longer need these + } +} + +/* + * (Job page only) Helper function to compute the absolute + * position of the specified element in our graph. + */ +function getAbsolutePosition(d3selection) { + if (d3selection.empty()) { + throw "Attempted to get absolute position of an empty selection."; + } + var obj = d3selection; + var _x = toFloat(obj.attr("x")) || 0; + var _y = toFloat(obj.attr("y")) || 0; + while (!obj.empty()) { + var transformText = obj.attr("transform"); + if (transformText) { + var translate = d3.transform(transformText).translate; + _x += toFloat(translate[0]); + _y += toFloat(translate[1]); + } + // Climb upwards to find how our parents are translated + obj = d3.select(obj.node().parentNode); + // Stop when we've reached the graph container itself + if (obj.node() == graphContainer().node()) { + break; + } + } + return { x: _x, y: _y }; +} + +/* (Job page only) Helper function to connect two RDDs with a curved edge. */ +function connectRDDs(fromRDDId, toRDDId, edgesContainer, svgContainer) { + var fromNodeId = VizConstants.nodePrefix + fromRDDId; + var toNodeId = VizConstants.nodePrefix + toRDDId; + var fromPos = getAbsolutePosition(svgContainer.select("g." + fromNodeId)); + var toPos = getAbsolutePosition(svgContainer.select("g." + toNodeId)); + + // On the job page, RDDs are rendered as dots (circles). When rendering the path, + // we need to account for the radii of these circles. Otherwise the arrow heads + // will bleed into the circle itself. + var delta = toFloat(svgContainer + .select("g.node." + toNodeId) + .select("circle") + .attr("r")); + if (fromPos.x < toPos.x) { + fromPos.x += delta; + toPos.x -= delta; + } else if (fromPos.x > toPos.x) { + fromPos.x -= delta; + toPos.x += delta; + } + + var points; + if (fromPos.y == toPos.y) { + // If they are on the same rank, curve the middle part of the edge + // upward a little to avoid interference with things in between + // e.g. _______ + // _____/ \_____ + points = [ + [fromPos.x, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.2, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.3, fromPos.y - 20], + [fromPos.x + (toPos.x - fromPos.x) * 0.7, fromPos.y - 20], + [fromPos.x + (toPos.x - fromPos.x) * 0.8, toPos.y], + [toPos.x, toPos.y] + ]; + } else { + // Otherwise, draw a curved edge that flattens out on both ends + // e.g. _____ + // / + // | + // _____/ + points = [ + [fromPos.x, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.4, fromPos.y], + [fromPos.x + (toPos.x - fromPos.x) * 0.6, toPos.y], + [toPos.x, toPos.y] + ]; + } + + var line = d3.svg.line().interpolate("basis"); + edgesContainer.append("path").datum(points).attr("d", line); +} + +/* (Job page only) Helper function to add tooltips for RDDs. */ +function addTooltipsForRDDs(svgContainer) { + svgContainer.selectAll("g.node").each(function() { + var node = d3.select(this); + var tooltipText = node.attr("name"); + if (tooltipText) { + node.select("circle") + .attr("data-toggle", "tooltip") + .attr("data-placement", "bottom") + .attr("title", tooltipText); + } + // Link tooltips for all nodes that belong to the same RDD + node.on("mouseenter", function() { triggerTooltipForRDD(node, true); }); + node.on("mouseleave", function() { triggerTooltipForRDD(node, false); }); + }); + + $("[data-toggle=tooltip]") + .filter("g.node circle") + .tooltip({ container: "body", trigger: "manual" }); +} + +/* + * (Job page only) Helper function to show or hide tooltips for all nodes + * in the graph that refer to the same RDD the specified node represents. + */ +function triggerTooltipForRDD(d3node, show) { + var classes = d3node.node().classList; + for (var i = 0; i < classes.length; i++) { + var clazz = classes[i]; + var isRDDClass = clazz.indexOf(VizConstants.nodePrefix) == 0; + if (isRDDClass) { + graphContainer().selectAll("g." + clazz).each(function() { + var circle = d3.select(this).select("circle").node(); + var showOrHide = show ? "show" : "hide"; + $(circle).tooltip(showOrHide); + }); + } + } +} + +/* Helper function to convert attributes to numeric values. */ +function toFloat(f) { + if (f) { + return parseFloat(f.toString().replace(/px$/, "")); + } else { + return f; + } +} + diff --git a/dpark/web/ui/static/js/dagre-d3.min.js b/dpark/web/ui/static/js/dagre-d3.min.js new file mode 100644 index 00000000..6fe8136c --- /dev/null +++ b/dpark/web/ui/static/js/dagre-d3.min.js @@ -0,0 +1,30 @@ +/* This is a custom version of dagre-d3 on top of v0.4.3. The full list of commits can be found at http://github.com/andrewor14/dagre-d3/ */ +!function(e){if("object"==typeof exports&&"undefined"!=typeof module)module.exports=e();else if("function"==typeof define&&define.amd)define([],e);else{var f;"undefined"!=typeof window?f=window:"undefined"!=typeof global?f=global:"undefined"!=typeof self&&(f=self),f.dagreD3=e()}}(function(){var define,module,exports;return function e(t,n,r){function s(o,u){if(!n[o]){if(!t[o]){var a=typeof require=="function"&&require;if(!u&&a)return a(o,!0);if(i)return i(o,!0);var f=new Error("Cannot find module '"+o+"'");throw f.code="MODULE_NOT_FOUND",f}var l=n[o]={exports:{}};t[o][0].call(l.exports,function(e){var n=t[o][1][e];return s(n?n:e)},l,l.exports,e,t,n,r)}return n[o].exports}var i=typeof require=="function"&&require;for(var o=0;o0}},{}],14:[function(require,module,exports){module.exports=intersectNode;function intersectNode(node,point){return node.intersect(point)}},{}],15:[function(require,module,exports){var intersectLine=require("./intersect-line");module.exports=intersectPolygon;function intersectPolygon(node,polyPoints,point){var x1=node.x;var y1=node.y;var intersections=[];var minX=Number.POSITIVE_INFINITY,minY=Number.POSITIVE_INFINITY;polyPoints.forEach(function(entry){minX=Math.min(minX,entry.x);minY=Math.min(minY,entry.y)});var left=x1-node.width/2-minX;var top=y1-node.height/2-minY;for(var i=0;i1){intersections.sort(function(p,q){var pdx=p.x-point.x,pdy=p.y-point.y,distp=Math.sqrt(pdx*pdx+pdy*pdy),qdx=q.x-point.x,qdy=q.y-point.y,distq=Math.sqrt(qdx*qdx+qdy*qdy);return distpMath.abs(dx)*h){if(dy<0){h=-h}sx=dy===0?0:h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=dx===0?0:w*dy/dx}return{x:x+sx,y:y+sy}}},{}],17:[function(require,module,exports){var util=require("../util");module.exports=addHtmlLabel;function addHtmlLabel(root,node){var fo=root.append("foreignObject").attr("width","100000");var div=fo.append("xhtml:div");var label=node.label;switch(typeof label){case"function":div.insert(label);break;case"object":div.insert(function(){return label});break;default:div.html(label)}util.applyStyle(div,node.labelStyle);div.style("display","inline-block");div.style("white-space","nowrap");var w,h;div.each(function(){w=this.clientWidth;h=this.clientHeight});fo.attr("width",w).attr("height",h);return fo}},{"../util":25}],18:[function(require,module,exports){var addTextLabel=require("./add-text-label"),addHtmlLabel=require("./add-html-label");module.exports=addLabel;function addLabel(root,node){var label=node.label;var labelSvg=root.append("g");if(typeof label!=="string"||node.labelType==="html"){addHtmlLabel(labelSvg,node)}else{addTextLabel(labelSvg,node)}var labelBBox=labelSvg.node().getBBox();labelSvg.attr("transform","translate("+-labelBBox.width/2+","+-labelBBox.height/2+")");return labelSvg}},{"./add-html-label":17,"./add-text-label":19}],19:[function(require,module,exports){var util=require("../util");module.exports=addTextLabel;function addTextLabel(root,node){var domNode=root.append("text");var lines=processEscapeSequences(node.label).split("\n");for(var i=0;imaxPadding){maxPadding=child.paddingTop}}return maxPadding}function getRank(g,v){var maxRank=0;var children=g.children(v);for(var i=0;imaxRank){maxRank=thisRank}}return maxRank}function orderByRank(g,nodes){return nodes.sort(function(x,y){return getRank(g,x)-getRank(g,y)})}function edgeToId(e){return escapeId(e.v)+":"+escapeId(e.w)+":"+escapeId(e.name)}var ID_DELIM=/:/g;function escapeId(str){return str?String(str).replace(ID_DELIM,"\\:"):""}function applyStyle(dom,styleFn){if(styleFn){dom.attr("style",styleFn)}}function applyClass(dom,classFn,otherClasses){if(classFn){dom.attr("class",classFn).attr("class",otherClasses+" "+dom.attr("class"))}}function applyTransition(selection,g){var graph=g.graph();if(_.isPlainObject(graph)){var transition=graph.transition;if(_.isFunction(transition)){return transition(selection)}}return selection}},{"./lodash":20}],26:[function(require,module,exports){module.exports="0.4.4-pre"},{}],27:[function(require,module,exports){module.exports={graphlib:require("./lib/graphlib"),layout:require("./lib/layout"),debug:require("./lib/debug"),util:{time:require("./lib/util").time,notime:require("./lib/util").notime},version:require("./lib/version")}},{"./lib/debug":32,"./lib/graphlib":33,"./lib/layout":35,"./lib/util":55,"./lib/version":56}],28:[function(require,module,exports){"use strict";var _=require("./lodash"),greedyFAS=require("./greedy-fas");module.exports={run:run,undo:undo};function run(g){var fas=g.graph().acyclicer==="greedy"?greedyFAS(g,weightFn(g)):dfsFAS(g);_.each(fas,function(e){var label=g.edge(e);g.removeEdge(e);label.forwardName=e.name;label.reversed=true;g.setEdge(e.w,e.v,label,_.uniqueId("rev"))});function weightFn(g){return function(e){return g.edge(e).weight}}}function dfsFAS(g){var fas=[],stack={},visited={};function dfs(v){if(_.has(visited,v)){return}visited[v]=true;stack[v]=true;_.each(g.outEdges(v),function(e){if(_.has(stack,e.w)){fas.push(e)}else{dfs(e.w)}});delete stack[v]}_.each(g.nodes(),dfs);return fas}function undo(g){_.each(g.edges(),function(e){var label=g.edge(e);if(label.reversed){g.removeEdge(e);var forwardName=label.forwardName;delete label.reversed;delete label.forwardName;g.setEdge(e.w,e.v,label,forwardName)}})}},{"./greedy-fas":34,"./lodash":36}],29:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports=addBorderSegments;function addBorderSegments(g){function dfs(v){var children=g.children(v),node=g.node(v);if(children.length){_.each(children,dfs)}if(_.has(node,"minRank")){node.borderLeft=[];node.borderRight=[];for(var rank=node.minRank,maxRank=node.maxRank+1;rank0;--i){entry=buckets[i].dequeue();if(entry){results=results.concat(removeNode(g,buckets,zeroIdx,entry,true));break}}}}return results}function removeNode(g,buckets,zeroIdx,entry,collectPredecessors){var results=collectPredecessors?[]:undefined;_.each(g.inEdges(entry.v),function(edge){var weight=g.edge(edge),uEntry=g.node(edge.v);if(collectPredecessors){results.push({v:edge.v,w:edge.w})}uEntry.out-=weight;assignBucket(buckets,zeroIdx,uEntry)});_.each(g.outEdges(entry.v),function(edge){var weight=g.edge(edge),w=edge.w,wEntry=g.node(w);wEntry["in"]-=weight;assignBucket(buckets,zeroIdx,wEntry)});g.removeNode(entry.v);return results}function buildState(g,weightFn){var fasGraph=new Graph,maxIn=0,maxOut=0;_.each(g.nodes(),function(v){fasGraph.setNode(v,{v:v,"in":0,out:0})});_.each(g.edges(),function(e){var prevWeight=fasGraph.edge(e.v,e.w)||0,weight=weightFn(e),edgeWeight=prevWeight+weight;fasGraph.setEdge(e.v,e.w,edgeWeight);maxOut=Math.max(maxOut,fasGraph.node(e.v).out+=weight);maxIn=Math.max(maxIn,fasGraph.node(e.w)["in"]+=weight)});var buckets=_.range(maxOut+maxIn+3).map(function(){return new List});var zeroIdx=maxIn+1;_.each(fasGraph.nodes(),function(v){assignBucket(buckets,zeroIdx,fasGraph.node(v))});return{graph:fasGraph,buckets:buckets,zeroIdx:zeroIdx}}function assignBucket(buckets,zeroIdx,entry){if(!entry.out){buckets[0].enqueue(entry)}else if(!entry["in"]){buckets[buckets.length-1].enqueue(entry)}else{buckets[entry.out-entry["in"]+zeroIdx].enqueue(entry)}}},{"./data/list":31,"./graphlib":33,"./lodash":36}],35:[function(require,module,exports){"use strict";var _=require("./lodash"),acyclic=require("./acyclic"),normalize=require("./normalize"),rank=require("./rank"),normalizeRanks=require("./util").normalizeRanks,parentDummyChains=require("./parent-dummy-chains"),removeEmptyRanks=require("./util").removeEmptyRanks,nestingGraph=require("./nesting-graph"),addBorderSegments=require("./add-border-segments"),coordinateSystem=require("./coordinate-system"),order=require("./order"),position=require("./position"),util=require("./util"),Graph=require("./graphlib").Graph;module.exports=layout;function layout(g,opts){var time=opts&&opts.debugTiming?util.time:util.notime;time("layout",function(){var layoutGraph=time(" buildLayoutGraph",function(){return buildLayoutGraph(g)});time(" runLayout",function(){runLayout(layoutGraph,time)});time(" updateInputGraph",function(){updateInputGraph(g,layoutGraph)})})}function runLayout(g,time){time(" makeSpaceForEdgeLabels",function(){makeSpaceForEdgeLabels(g)});time(" removeSelfEdges",function(){removeSelfEdges(g)});time(" acyclic",function(){acyclic.run(g)});time(" nestingGraph.run",function(){nestingGraph.run(g)});time(" rank",function(){rank(util.asNonCompoundGraph(g))});time(" injectEdgeLabelProxies",function(){injectEdgeLabelProxies(g)});time(" removeEmptyRanks",function(){removeEmptyRanks(g)});time(" nestingGraph.cleanup",function(){nestingGraph.cleanup(g)});time(" normalizeRanks",function(){normalizeRanks(g)});time(" assignRankMinMax",function(){assignRankMinMax(g)});time(" removeEdgeLabelProxies",function(){removeEdgeLabelProxies(g)});time(" normalize.run",function(){ +normalize.run(g)});time(" parentDummyChains",function(){parentDummyChains(g)});time(" addBorderSegments",function(){addBorderSegments(g)});time(" order",function(){order(g)});time(" insertSelfEdges",function(){insertSelfEdges(g)});time(" adjustCoordinateSystem",function(){coordinateSystem.adjust(g)});time(" position",function(){position(g)});time(" positionSelfEdges",function(){positionSelfEdges(g)});time(" removeBorderNodes",function(){removeBorderNodes(g)});time(" normalize.undo",function(){normalize.undo(g)});time(" fixupEdgeLabelCoords",function(){fixupEdgeLabelCoords(g)});time(" undoCoordinateSystem",function(){coordinateSystem.undo(g)});time(" translateGraph",function(){translateGraph(g)});time(" assignNodeIntersects",function(){assignNodeIntersects(g)});time(" reversePoints",function(){reversePointsForReversedEdges(g)});time(" acyclic.undo",function(){acyclic.undo(g)})}function updateInputGraph(inputGraph,layoutGraph){_.each(inputGraph.nodes(),function(v){var inputLabel=inputGraph.node(v),layoutLabel=layoutGraph.node(v);if(inputLabel){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y;if(layoutGraph.children(v).length){inputLabel.width=layoutLabel.width;inputLabel.height=layoutLabel.height}}});_.each(inputGraph.edges(),function(e){var inputLabel=inputGraph.edge(e),layoutLabel=layoutGraph.edge(e);inputLabel.points=layoutLabel.points;if(_.has(layoutLabel,"x")){inputLabel.x=layoutLabel.x;inputLabel.y=layoutLabel.y}});inputGraph.graph().width=layoutGraph.graph().width;inputGraph.graph().height=layoutGraph.graph().height}var graphNumAttrs=["nodesep","edgesep","ranksep","marginx","marginy"],graphDefaults={ranksep:50,edgesep:20,nodesep:50,rankdir:"tb"},graphAttrs=["acyclicer","ranker","rankdir","align"],nodeNumAttrs=["width","height"],nodeDefaults={width:0,height:0},edgeNumAttrs=["minlen","weight","width","height","labeloffset"],edgeDefaults={minlen:1,weight:1,width:0,height:0,labeloffset:10,labelpos:"r"},edgeAttrs=["labelpos"];function buildLayoutGraph(inputGraph){var g=new Graph({multigraph:true,compound:true}),graph=canonicalize(inputGraph.graph());g.setGraph(_.merge({},graphDefaults,selectNumberAttrs(graph,graphNumAttrs),_.pick(graph,graphAttrs)));_.each(inputGraph.nodes(),function(v){var node=canonicalize(inputGraph.node(v));g.setNode(v,_.defaults(selectNumberAttrs(node,nodeNumAttrs),nodeDefaults));g.setParent(v,inputGraph.parent(v))});_.each(inputGraph.edges(),function(e){var edge=canonicalize(inputGraph.edge(e));g.setEdge(e,_.merge({},edgeDefaults,selectNumberAttrs(edge,edgeNumAttrs),_.pick(edge,edgeAttrs)))});return g}function makeSpaceForEdgeLabels(g){var graph=g.graph();graph.ranksep/=2;_.each(g.edges(),function(e){var edge=g.edge(e);edge.minlen*=2;if(edge.labelpos.toLowerCase()!=="c"){if(graph.rankdir==="TB"||graph.rankdir==="BT"){edge.width+=edge.labeloffset}else{edge.height+=edge.labeloffset}}})}function injectEdgeLabelProxies(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.width&&edge.height){var v=g.node(e.v),w=g.node(e.w),label={rank:(w.rank-v.rank)/2+v.rank,e:e};util.addDummyNode(g,"edge-proxy",label,"_ep")}})}function assignRankMinMax(g){var maxRank=0;_.each(g.nodes(),function(v){var node=g.node(v);if(node.borderTop){node.minRank=g.node(node.borderTop).rank;node.maxRank=g.node(node.borderBottom).rank;maxRank=_.max(maxRank,node.maxRank)}});g.graph().maxRank=maxRank}function removeEdgeLabelProxies(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="edge-proxy"){g.edge(node.e).labelRank=node.rank;g.removeNode(v)}})}function translateGraph(g){var minX=Number.POSITIVE_INFINITY,maxX=0,minY=Number.POSITIVE_INFINITY,maxY=0,graphLabel=g.graph(),marginX=graphLabel.marginx||0,marginY=graphLabel.marginy||0;function getExtremes(attrs){var x=attrs.x,y=attrs.y,w=attrs.width,h=attrs.height;minX=Math.min(minX,x-w/2);maxX=Math.max(maxX,x+w/2);minY=Math.min(minY,y-h/2);maxY=Math.max(maxY,y+h/2)}_.each(g.nodes(),function(v){getExtremes(g.node(v))});_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){getExtremes(edge)}});minX-=marginX;minY-=marginY;_.each(g.nodes(),function(v){var node=g.node(v);node.x-=minX;node.y-=minY});_.each(g.edges(),function(e){var edge=g.edge(e);_.each(edge.points,function(p){p.x-=minX;p.y-=minY});if(_.has(edge,"x")){edge.x-=minX}if(_.has(edge,"y")){edge.y-=minY}});graphLabel.width=maxX-minX+marginX;graphLabel.height=maxY-minY+marginY}function assignNodeIntersects(g){_.each(g.edges(),function(e){var edge=g.edge(e),nodeV=g.node(e.v),nodeW=g.node(e.w),p1,p2;if(!edge.points){edge.points=[];p1=nodeW;p2=nodeV}else{p1=edge.points[0];p2=edge.points[edge.points.length-1]}edge.points.unshift(util.intersectRect(nodeV,p1));edge.points.push(util.intersectRect(nodeW,p2))})}function fixupEdgeLabelCoords(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(_.has(edge,"x")){if(edge.labelpos==="l"||edge.labelpos==="r"){edge.width-=edge.labeloffset}switch(edge.labelpos){case"l":edge.x-=edge.width/2+edge.labeloffset;break;case"r":edge.x+=edge.width/2+edge.labeloffset;break}}})}function reversePointsForReversedEdges(g){_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.reversed){edge.points.reverse()}})}function removeBorderNodes(g){_.each(g.nodes(),function(v){if(g.children(v).length){var node=g.node(v),t=g.node(node.borderTop),b=g.node(node.borderBottom),l=g.node(_.last(node.borderLeft)),r=g.node(_.last(node.borderRight));node.width=Math.abs(r.x-l.x);node.height=Math.abs(b.y-t.y);node.x=l.x+node.width/2;node.y=t.y+node.height/2}});_.each(g.nodes(),function(v){if(g.node(v).dummy==="border"){g.removeNode(v)}})}function removeSelfEdges(g){_.each(g.edges(),function(e){if(e.v===e.w){var node=g.node(e.v);if(!node.selfEdges){node.selfEdges=[]}node.selfEdges.push({e:e,label:g.edge(e)});g.removeEdge(e)}})}function insertSelfEdges(g){var layers=util.buildLayerMatrix(g);_.each(layers,function(layer){var orderShift=0;_.each(layer,function(v,i){var node=g.node(v);node.order=i+orderShift;_.each(node.selfEdges,function(selfEdge){util.addDummyNode(g,"selfedge",{width:selfEdge.label.width,height:selfEdge.label.height,rank:node.rank,order:i+ ++orderShift,e:selfEdge.e,label:selfEdge.label},"_se")});delete node.selfEdges})})}function positionSelfEdges(g){_.each(g.nodes(),function(v){var node=g.node(v);if(node.dummy==="selfedge"){var selfNode=g.node(node.e.v),x=selfNode.x+selfNode.width/2,y=selfNode.y,dx=node.x-x,dy=selfNode.height/2;g.setEdge(node.e,node.label);g.removeNode(v);node.label.points=[{x:x+2*dx/3,y:y-dy},{x:x+5*dx/6,y:y-dy},{x:x+dx,y:y},{x:x+5*dx/6,y:y+dy},{x:x+2*dx/3,y:y+dy}];node.label.x=node.x;node.label.y=node.y}})}function selectNumberAttrs(obj,attrs){return _.mapValues(_.pick(obj,attrs),Number)}function canonicalize(attrs){var newAttrs={};_.each(attrs,function(v,k){newAttrs[k.toLowerCase()]=v});return newAttrs}},{"./acyclic":28,"./add-border-segments":29,"./coordinate-system":30,"./graphlib":33,"./lodash":36,"./nesting-graph":37,"./normalize":38,"./order":43,"./parent-dummy-chains":48,"./position":50,"./rank":52,"./util":55}],36:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],37:[function(require,module,exports){var _=require("./lodash"),util=require("./util");module.exports={run:run,cleanup:cleanup};function run(g){var root=util.addDummyNode(g,"root",{},"_root"),depths=treeDepths(g),height=_.max(depths)-1,nodeSep=2*height+1;g.graph().nestingRoot=root;_.each(g.edges(),function(e){g.edge(e).minlen*=nodeSep});var weight=sumWeights(g)+1;_.each(g.children(),function(child){dfs(g,root,nodeSep,weight,height,depths,child)});g.graph().nodeRankFactor=nodeSep}function dfs(g,root,nodeSep,weight,height,depths,v){var children=g.children(v);if(!children.length){if(v!==root){g.setEdge(root,v,{weight:0,minlen:nodeSep})}return}var top=util.addBorderNode(g,"_bt"),bottom=util.addBorderNode(g,"_bb"),label=g.node(v);g.setParent(top,v);label.borderTop=top;g.setParent(bottom,v);label.borderBottom=bottom;_.each(children,function(child){dfs(g,root,nodeSep,weight,height,depths,child);var childNode=g.node(child),childTop=childNode.borderTop?childNode.borderTop:child,childBottom=childNode.borderBottom?childNode.borderBottom:child,thisWeight=childNode.borderTop?weight:2*weight,minlen=childTop!==childBottom?1:height-depths[v]+1;g.setEdge(top,childTop,{weight:thisWeight,minlen:minlen,nestingEdge:true});g.setEdge(childBottom,bottom,{weight:thisWeight,minlen:minlen,nestingEdge:true})});if(!g.parent(v)){g.setEdge(root,top,{weight:0,minlen:height+depths[v]})}}function treeDepths(g){var depths={};function dfs(v,depth){var children=g.children(v);if(children&&children.length){_.each(children,function(child){dfs(child,depth+1)})}depths[v]=depth}_.each(g.children(),function(v){dfs(v,1)});return depths}function sumWeights(g){return _.reduce(g.edges(),function(acc,e){return acc+g.edge(e).weight},0)}function cleanup(g){var graphLabel=g.graph();g.removeNode(graphLabel.nestingRoot);delete graphLabel.nestingRoot;_.each(g.edges(),function(e){var edge=g.edge(e);if(edge.nestingEdge){g.removeEdge(e)}})}},{"./lodash":36,"./util":55}],38:[function(require,module,exports){"use strict";var _=require("./lodash"),util=require("./util");module.exports={run:run,undo:undo};function run(g){g.graph().dummyChains=[];_.each(g.edges(),function(edge){normalizeEdge(g,edge)})}function normalizeEdge(g,e){var v=e.v,vRank=g.node(v).rank,w=e.w,wRank=g.node(w).rank,name=e.name,edgeLabel=g.edge(e),labelRank=edgeLabel.labelRank;if(wRank===vRank+1)return;g.removeEdge(e);var dummy,attrs,i;for(i=0,++vRank;vRank0){if(index%2){weightSum+=tree[index+1]}index=index-1>>1;tree[index]+=entry.weight}cc+=entry.weight*weightSum}));return cc}},{"../lodash":36}],43:[function(require,module,exports){"use strict";var _=require("../lodash"),initOrder=require("./init-order"),crossCount=require("./cross-count"),sortSubgraph=require("./sort-subgraph"),buildLayerGraph=require("./build-layer-graph"),addSubgraphConstraints=require("./add-subgraph-constraints"),Graph=require("../graphlib").Graph,util=require("../util");module.exports=order;function order(g){var maxRank=util.maxRank(g),downLayerGraphs=buildLayerGraphs(g,_.range(1,maxRank+1),"inEdges"),upLayerGraphs=buildLayerGraphs(g,_.range(maxRank-1,-1,-1),"outEdges");var layering=initOrder(g);assignOrder(g,layering);var bestCC=Number.POSITIVE_INFINITY,best;for(var i=0,lastBest=0;lastBest<4;++i,++lastBest){sweepLayerGraphs(i%2?downLayerGraphs:upLayerGraphs,i%4>=2);layering=util.buildLayerMatrix(g);var cc=crossCount(g,layering);if(cc=vEntry.barycenter){mergeEntries(vEntry,uEntry)}}}function handleOut(vEntry){return function(wEntry){wEntry["in"].push(vEntry);if(--wEntry.indegree===0){sourceSet.push(wEntry)}}}while(sourceSet.length){var entry=sourceSet.pop();entries.push(entry);_.each(entry["in"].reverse(),handleIn(entry));_.each(entry.out,handleOut(entry))}return _.chain(entries).filter(function(entry){return!entry.merged}).map(function(entry){return _.pick(entry,["vs","i","barycenter","weight"])}).value()}function mergeEntries(target,source){var sum=0,weight=0;if(target.weight){sum+=target.barycenter*target.weight;weight+=target.weight}if(source.weight){sum+=source.barycenter*source.weight;weight+=source.weight}target.vs=source.vs.concat(target.vs);target.barycenter=sum/weight;target.weight=weight;target.i=Math.min(source.i,target.i);source.merged=true}},{"../lodash":36}],46:[function(require,module,exports){var _=require("../lodash"),barycenter=require("./barycenter"),resolveConflicts=require("./resolve-conflicts"),sort=require("./sort");module.exports=sortSubgraph;function sortSubgraph(g,v,cg,biasRight){var movable=g.children(v),node=g.node(v),bl=node?node.borderLeft:undefined,br=node?node.borderRight:undefined,subgraphs={};if(bl){movable=_.filter(movable,function(w){return w!==bl&&w!==br})}var barycenters=barycenter(g,movable);_.each(barycenters,function(entry){if(g.children(entry.v).length){var subgraphResult=sortSubgraph(g,entry.v,cg,biasRight);subgraphs[entry.v]=subgraphResult;if(_.has(subgraphResult,"barycenter")){mergeBarycenters(entry,subgraphResult)}}});var entries=resolveConflicts(barycenters,cg);expandSubgraphs(entries,subgraphs);var result=sort(entries,biasRight);if(bl){result.vs=_.flatten([bl,result.vs,br],true);if(g.predecessors(bl).length){var blPred=g.node(g.predecessors(bl)[0]),brPred=g.node(g.predecessors(br)[0]);if(!_.has(result,"barycenter")){result.barycenter=0;result.weight=0}result.barycenter=(result.barycenter*result.weight+blPred.order+brPred.order)/(result.weight+2);result.weight+=2}}return result}function expandSubgraphs(entries,subgraphs){_.each(entries,function(entry){entry.vs=_.flatten(entry.vs.map(function(v){if(subgraphs[v]){return subgraphs[v].vs}return v}),true)})}function mergeBarycenters(target,other){if(!_.isUndefined(target.barycenter)){target.barycenter=(target.barycenter*target.weight+other.barycenter*other.weight)/(target.weight+other.weight);target.weight+=other.weight}else{target.barycenter=other.barycenter;target.weight=other.weight}}},{"../lodash":36,"./barycenter":40,"./resolve-conflicts":45,"./sort":47}],47:[function(require,module,exports){var _=require("../lodash"),util=require("../util");module.exports=sort;function sort(entries,biasRight){var parts=util.partition(entries,function(entry){return _.has(entry,"barycenter")});var sortable=parts.lhs,unsortable=_.sortBy(parts.rhs,function(entry){return-entry.i}),vs=[],sum=0,weight=0,vsIndex=0;sortable.sort(compareWithBias(!!biasRight));vsIndex=consumeUnsortable(vs,unsortable,vsIndex);_.each(sortable,function(entry){vsIndex+=entry.vs.length;vs.push(entry.vs);sum+=entry.barycenter*entry.weight;weight+=entry.weight;vsIndex=consumeUnsortable(vs,unsortable,vsIndex)});var result={vs:_.flatten(vs,true)};if(weight){result.barycenter=sum/weight;result.weight=weight}return result}function consumeUnsortable(vs,unsortable,index){var last;while(unsortable.length&&(last=_.last(unsortable)).i<=index){unsortable.pop();vs.push(last.vs);index++}return index}function compareWithBias(bias){return function(entryV,entryW){if(entryV.barycenterentryW.barycenter){return 1}return!bias?entryV.i-entryW.i:entryW.i-entryV.i}}},{"../lodash":36,"../util":55}],48:[function(require,module,exports){var _=require("./lodash");module.exports=parentDummyChains;function parentDummyChains(g){var postorderNums=postorder(g);_.each(g.graph().dummyChains,function(v){var node=g.node(v),edgeObj=node.edgeObj,pathData=findPath(g,postorderNums,edgeObj.v,edgeObj.w),path=pathData.path,lca=pathData.lca,pathIdx=0,pathV=path[pathIdx],ascending=true;while(v!==edgeObj.w){node=g.node(v);if(ascending){while((pathV=path[pathIdx])!==lca&&g.node(pathV).maxRanklow||lim>postorderNums[parent].lim));lca=parent;parent=w;while((parent=g.parent(parent))!==lca){wPath.push(parent)}return{path:vPath.concat(wPath.reverse()),lca:lca}}function postorder(g){var result={},lim=0;function dfs(v){var low=lim;_.each(g.children(v),dfs);result[v]={low:low,lim:lim++}}_.each(g.children(),dfs);return result}},{"./lodash":36}],49:[function(require,module,exports){"use strict";var _=require("../lodash"),Graph=require("../graphlib").Graph,util=require("../util");module.exports={positionX:positionX,findType1Conflicts:findType1Conflicts,findType2Conflicts:findType2Conflicts,addConflict:addConflict,hasConflict:hasConflict,verticalAlignment:verticalAlignment,horizontalCompaction:horizontalCompaction,alignCoordinates:alignCoordinates,findSmallestWidthAlignment:findSmallestWidthAlignment,balance:balance};function findType1Conflicts(g,layering){var conflicts={};function visitLayer(prevLayer,layer){var k0=0,scanPos=0,prevLayerLength=prevLayer.length,lastNode=_.last(layer);_.each(layer,function(v,i){var w=findOtherInnerSegmentNode(g,v),k1=w?g.node(w).order:prevLayerLength;if(w||v===lastNode){_.each(layer.slice(scanPos,i+1),function(scanNode){_.each(g.predecessors(scanNode),function(u){var uLabel=g.node(u),uPos=uLabel.order;if((uPosnextNorthBorder)){addConflict(conflicts,u,v)}})}})}function visitLayer(north,south){var prevNorthPos=-1,nextNorthPos,southPos=0;_.each(south,function(v,southLookahead){if(g.node(v).dummy==="border"){var predecessors=g.predecessors(v);if(predecessors.length){nextNorthPos=g.node(predecessors[0]).order;scan(south,southPos,southLookahead,prevNorthPos,nextNorthPos);southPos=southLookahead;prevNorthPos=nextNorthPos}}scan(south,southPos,south.length,nextNorthPos,north.length)});return south}_.reduce(layering,visitLayer);return conflicts}function findOtherInnerSegmentNode(g,v){if(g.node(v).dummy){return _.find(g.predecessors(v),function(u){return g.node(u).dummy})}}function addConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}var conflictsV=conflicts[v];if(!conflictsV){conflicts[v]=conflictsV={}}conflictsV[w]=true}function hasConflict(conflicts,v,w){if(v>w){var tmp=v;v=w;w=tmp}return _.has(conflicts[v],w)}function verticalAlignment(g,layering,conflicts,neighborFn){var root={},align={},pos={};_.each(layering,function(layer){_.each(layer,function(v,order){root[v]=v;align[v]=v;pos[v]=order})});_.each(layering,function(layer){var prevIdx=-1;_.each(layer,function(v){var ws=neighborFn(v);if(ws.length){ws=_.sortBy(ws,function(w){return pos[w]});var mp=(ws.length-1)/2;for(var i=Math.floor(mp),il=Math.ceil(mp);i<=il;++i){var w=ws[i];if(align[v]===v&&prevIdxwLabel.lim){tailLabel=wLabel;flip=true}var candidates=_.filter(g.edges(),function(edge){return flip===isDescendant(t,t.node(edge.v),tailLabel)&&flip!==isDescendant(t,t.node(edge.w),tailLabel)});return _.min(candidates,function(edge){return slack(g,edge)})}function exchangeEdges(t,g,e,f){var v=e.v,w=e.w;t.removeEdge(v,w);t.setEdge(f.v,f.w,{});initLowLimValues(t);initCutValues(t,g);updateRanks(t,g)}function updateRanks(t,g){var root=_.find(t.nodes(),function(v){return!g.node(v).parent}),vs=preorder(t,root);vs=vs.slice(1);_.each(vs,function(v){var parent=t.node(v).parent,edge=g.edge(v,parent),flipped=false;if(!edge){edge=g.edge(parent,v);flipped=true}g.node(v).rank=g.node(parent).rank+(flipped?edge.minlen:-edge.minlen)})}function isTreeEdge(tree,u,v){return tree.hasEdge(u,v)}function isDescendant(tree,vLabel,rootLabel){return rootLabel.low<=vLabel.lim&&vLabel.lim<=rootLabel.lim}},{"../graphlib":33,"../lodash":36,"../util":55,"./feasible-tree":51,"./util":54}],54:[function(require,module,exports){"use strict";var _=require("../lodash");module.exports={longestPath:longestPath,slack:slack};function longestPath(g){var visited={};function dfs(v){var label=g.node(v);if(_.has(visited,v)){return label.rank}visited[v]=true;var rank=_.min(_.map(g.outEdges(v),function(e){return dfs(e.w)-g.edge(e).minlen}));if(rank===Number.POSITIVE_INFINITY){rank=0}return label.rank=rank}_.each(g.sources(),dfs)}function slack(g,e){return g.node(e.w).rank-g.node(e.v).rank-g.edge(e).minlen}},{"../lodash":36}],55:[function(require,module,exports){"use strict";var _=require("./lodash"),Graph=require("./graphlib").Graph;module.exports={addDummyNode:addDummyNode,simplify:simplify,asNonCompoundGraph:asNonCompoundGraph,successorWeights:successorWeights,predecessorWeights:predecessorWeights,intersectRect:intersectRect,buildLayerMatrix:buildLayerMatrix,normalizeRanks:normalizeRanks,removeEmptyRanks:removeEmptyRanks,addBorderNode:addBorderNode,maxRank:maxRank,partition:partition,time:time,notime:notime};function addDummyNode(g,type,attrs,name){var v;do{v=_.uniqueId(name)}while(g.hasNode(v));attrs.dummy=type;g.setNode(v,attrs);return v}function simplify(g){var simplified=(new Graph).setGraph(g.graph());_.each(g.nodes(),function(v){simplified.setNode(v,g.node(v))});_.each(g.edges(),function(e){var simpleLabel=simplified.edge(e.v,e.w)||{weight:0,minlen:1},label=g.edge(e);simplified.setEdge(e.v,e.w,{weight:simpleLabel.weight+label.weight,minlen:Math.max(simpleLabel.minlen,label.minlen)})});return simplified}function asNonCompoundGraph(g){var simplified=new Graph({multigraph:g.isMultigraph()}).setGraph(g.graph());_.each(g.nodes(),function(v){if(!g.children(v).length){simplified.setNode(v,g.node(v))}});_.each(g.edges(),function(e){simplified.setEdge(e,g.edge(e))});return simplified}function successorWeights(g){var weightMap=_.map(g.nodes(),function(v){var sucs={};_.each(g.outEdges(v),function(e){sucs[e.w]=(sucs[e.w]||0)+g.edge(e).weight});return sucs});return _.zipObject(g.nodes(),weightMap)}function predecessorWeights(g){var weightMap=_.map(g.nodes(),function(v){var preds={};_.each(g.inEdges(v),function(e){preds[e.v]=(preds[e.v]||0)+g.edge(e).weight});return preds});return _.zipObject(g.nodes(),weightMap)}function intersectRect(rect,point){var x=rect.x;var y=rect.y;var dx=point.x-x;var dy=point.y-y;var w=rect.width/2;var h=rect.height/2;if(!dx&&!dy){throw new Error("Not possible to find intersection inside of the rectangle")}var sx,sy;if(Math.abs(dy)*w>Math.abs(dx)*h){if(dy<0){h=-h}sx=h*dx/dy;sy=h}else{if(dx<0){w=-w}sx=w;sy=w*dy/dx}return{x:x+sx,y:y+sy}}function buildLayerMatrix(g){var layering=_.map(_.range(maxRank(g)+1),function(){return[]});_.each(g.nodes(),function(v){var node=g.node(v),rank=node.rank;if(!_.isUndefined(rank)){layering[rank][node.order]=v}});return layering}function normalizeRanks(g){var min=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));_.each(g.nodes(),function(v){var node=g.node(v);if(_.has(node,"rank")){node.rank-=min}})}function removeEmptyRanks(g){var offset=_.min(_.map(g.nodes(),function(v){return g.node(v).rank}));var layers=[];_.each(g.nodes(),function(v){var rank=g.node(v).rank-offset;if(!_.has(layers,rank)){layers[rank]=[]}layers[rank].push(v)});var delta=0,nodeRankFactor=g.graph().nodeRankFactor;_.each(layers,function(vs,i){if(_.isUndefined(vs)&&i%nodeRankFactor!==0){--delta}else if(delta){_.each(vs,function(v){g.node(v).rank+=delta})}})}function addBorderNode(g,prefix,rank,order){var node={width:0,height:0};if(arguments.length>=4){node.rank=rank;node.order=order}return addDummyNode(g,"border",node,prefix)}function maxRank(g){return _.max(_.map(g.nodes(),function(v){var rank=g.node(v).rank;if(!_.isUndefined(rank)){return rank}}))}function partition(collection,fn){var result={lhs:[],rhs:[]};_.each(collection,function(value){if(fn(value)){result.lhs.push(value)}else{result.rhs.push(value)}});return result}function time(name,fn){var start=_.now();try{return fn()}finally{console.log(name+" time: "+(_.now()-start)+"ms")}}function notime(name,fn){return fn()}},{"./graphlib":33,"./lodash":36}],56:[function(require,module,exports){module.exports="0.7.1"},{}],57:[function(require,module,exports){var lib=require("./lib");module.exports={Graph:lib.Graph,json:require("./lib/json"),alg:require("./lib/alg"),version:lib.version}},{"./lib":73,"./lib/alg":64,"./lib/json":74}],58:[function(require,module,exports){var _=require("../lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{"../lodash":75}],59:[function(require,module,exports){var _=require("../lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{"../lodash":75}],60:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("../lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"../lodash":75,"./dijkstra":61}],61:[function(require,module,exports){var _=require("../lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":71,"../lodash":75}],62:[function(require,module,exports){var _=require("../lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"../lodash":75,"./tarjan":69}],63:[function(require,module,exports){var _=require("../lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":71,"../graph":72,"../lodash":75}],69:[function(require,module,exports){var _=require("../lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{"../lodash":75}],70:[function(require,module,exports){var _=require("../lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{"../lodash":75}],71:[function(require,module,exports){var _=require("../lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(){var v,w,name,value,valueSpecified=false;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}else{v=arguments[0];w=arguments[1];name=arguments[3];if(arguments.length>2){value=arguments[2];valueSpecified=true}}v=""+v;w=""+w;if(!_.isUndefined(name)){name=""+name}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{"./lodash":75}],73:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":72,"./version":76}],74:[function(require,module,exports){var _=require("./lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":72,"./lodash":75}],75:[function(require,module,exports){module.exports=require(20)},{"/Users/andrew/Documents/dev/dagre-d3/lib/lodash.js":20,lodash:77}],76:[function(require,module,exports){module.exports="1.0.1"},{}],77:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f \ufeff"+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){ +return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++index1?arguments[1]:{},peg$FAILED={},peg$startRuleFunctions={start:peg$parsestart,graphStmt:peg$parsegraphStmt},peg$startRuleFunction=peg$parsestart,peg$c0=[],peg$c1=peg$FAILED,peg$c2=null,peg$c3="{",peg$c4={type:"literal",value:"{",description:'"{"'},peg$c5="}",peg$c6={type:"literal",value:"}",description:'"}"'},peg$c7=function(strict,type,id,stmts){return{type:type,id:id,strict:strict!==null,stmts:stmts}},peg$c8=";",peg$c9={type:"literal",value:";",description:'";"'},peg$c10=function(first,rest){var result=[first];for(var i=0;i",description:'"->"'},peg$c33=function(rhs,rest){var result=[rhs];if(rest){for(var i=0;ipos){peg$cachedPos=0;peg$cachedPosDetails={line:1,column:1,seenCR:false}}advance(peg$cachedPosDetails,peg$cachedPos,pos);peg$cachedPos=pos}return peg$cachedPosDetails}function peg$fail(expected){if(peg$currPospeg$maxFailPos){peg$maxFailPos=peg$currPos;peg$maxFailExpected=[]}peg$maxFailExpected.push(expected)}function peg$buildException(message,expected,pos){function cleanupExpected(expected){var i=1;expected.sort(function(a,b){if(a.descriptionb.description){return 1}else{return 0}});while(i1?expectedDescs.slice(0,-1).join(", ")+" or "+expectedDescs[expected.length-1]:expectedDescs[0];foundDesc=found?'"'+stringEscape(found)+'"':"end of input";return"Expected "+expectedDesc+" but "+foundDesc+" found."}var posDetails=peg$computePosDetails(pos),found=pospeg$currPos){s5=input.charAt(peg$currPos);peg$currPos++}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c110)}}if(s5!==peg$FAILED){s4=[s4,s5];s3=s4}else{peg$currPos=s3;s3=peg$c1}}else{peg$currPos=s3;s3=peg$c1}while(s3!==peg$FAILED){s2.push(s3);s3=peg$currPos;s4=peg$currPos;peg$silentFails++;if(input.substr(peg$currPos,2)===peg$c108){s5=peg$c108;peg$currPos+=2}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c109)}}peg$silentFails--;if(s5===peg$FAILED){s4=peg$c30}else{peg$currPos=s4;s4=peg$c1}if(s4!==peg$FAILED){if(input.length>peg$currPos){s5=input.charAt(peg$currPos);peg$currPos++}else{s5=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c110)}}if(s5!==peg$FAILED){s4=[s4,s5];s3=s4}else{peg$currPos=s3;s3=peg$c1}}else{peg$currPos=s3;s3=peg$c1}}if(s2!==peg$FAILED){if(input.substr(peg$currPos,2)===peg$c108){s3=peg$c108;peg$currPos+=2}else{s3=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c109)}}if(s3!==peg$FAILED){s1=[s1,s2,s3];s0=s1}else{peg$currPos=s0;s0=peg$c1}}else{peg$currPos=s0;s0=peg$c1}}else{peg$currPos=s0;s0=peg$c1}}peg$silentFails--;if(s0===peg$FAILED){s1=peg$FAILED;if(peg$silentFails===0){peg$fail(peg$c101)}}return s0}function peg$parse_(){var s0;s0=peg$parsewhitespace();if(s0===peg$FAILED){s0=peg$parsecomment()}return s0}var _=require("lodash");var directed;peg$result=peg$startRuleFunction();if(peg$result!==peg$FAILED&&peg$currPos===input.length){return peg$result}else{if(peg$result!==peg$FAILED&&peg$currPos":"--",writer=new Writer;if(!g.isMultigraph()){writer.write("strict ")}writer.writeLine((g.isDirected()?"digraph":"graph")+" {");writer.indent();var graphAttrs=g.graph();if(_.isObject(graphAttrs)){_.each(graphAttrs,function(v,k){writer.writeLine(id(k)+"="+id(v)+";")})}writeSubgraph(g,undefined,writer);g.edges().forEach(function(edge){writeEdge(g,edge,ec,writer)});writer.unindent();writer.writeLine("}");return writer.toString()}function writeSubgraph(g,v,writer){var children=g.isCompound()?g.children(v):g.nodes();_.each(children,function(w){if(!g.isCompound()||!g.children(w).length){writeNode(g,w,writer)}else{writer.writeLine("subgraph "+id(w)+" {");writer.indent();if(_.isObject(g.node(w))){_.map(g.node(w),function(val,key){writer.writeLine(id(key)+"="+id(val)+";")})}writeSubgraph(g,w,writer);writer.unindent();writer.writeLine("}")}})}function writeNode(g,v,writer){writer.write(id(v));writeAttrs(g.node(v),writer);writer.writeLine()}function writeEdge(g,edge,ec,writer){var v=edge.v,w=edge.w,attrs=g.edge(edge);writer.write(id(v)+" "+ec+" "+id(w));writeAttrs(attrs,writer);writer.writeLine()}function writeAttrs(attrs,writer){if(_.isObject(attrs)){var attrStrs=_.map(attrs,function(val,key){return id(key)+"="+id(val)});if(attrStrs.length){writer.write(" ["+attrStrs.join(",")+"]")}}}function id(obj){if(typeof obj==="number"||obj.toString().match(UNESCAPED_ID_PATTERN)){return obj}return'"'+obj.toString().replace(/"/g,'\\"')+'"'}function Writer(){this._indent="";this._content="";this._shouldIndent=true}Writer.prototype.INDENT=" ";Writer.prototype.indent=function(){this._indent+=this.INDENT};Writer.prototype.unindent=function(){this._indent=this._indent.slice(this.INDENT.length)};Writer.prototype.writeLine=function(line){this.write((line||"")+"\n");this._shouldIndent=true};Writer.prototype.write=function(str){if(this._shouldIndent){this._shouldIndent=false;this._content+=this._indent}this._content+=str};Writer.prototype.toString=function(){return this._content}},{lodash:28}],9:[function(require,module,exports){var _=require("lodash");module.exports=_.clone(require("./lib"));module.exports.json=require("./lib/json");module.exports.alg=require("./lib/alg")},{"./lib":25,"./lib/alg":16,"./lib/json":26,lodash:28}],10:[function(require,module,exports){var _=require("lodash");module.exports=components;function components(g){var visited={},cmpts=[],cmpt;function dfs(v){if(_.has(visited,v))return;visited[v]=true;cmpt.push(v);_.each(g.successors(v),dfs);_.each(g.predecessors(v),dfs)}_.each(g.nodes(),function(v){cmpt=[];dfs(v);if(cmpt.length){cmpts.push(cmpt)}});return cmpts}},{lodash:28}],11:[function(require,module,exports){var _=require("lodash");module.exports=dfs;function dfs(g,vs,order){if(!_.isArray(vs)){vs=[vs]}var acc=[],visited={};_.each(vs,function(v){if(!g.hasNode(v)){throw new Error("Graph does not have node: "+v)}doDfs(g,v,order==="post",visited,acc)});return acc}function doDfs(g,v,postorder,visited,acc){if(!_.has(visited,v)){visited[v]=true;if(!postorder){acc.push(v)}_.each(g.neighbors(v),function(w){doDfs(g,w,postorder,visited,acc)});if(postorder){acc.push(v)}}}},{lodash:28}],12:[function(require,module,exports){var dijkstra=require("./dijkstra"),_=require("lodash");module.exports=dijkstraAll;function dijkstraAll(g,weightFunc,edgeFunc){return _.transform(g.nodes(),function(acc,v){acc[v]=dijkstra(g,v,weightFunc,edgeFunc)},{})}},{"./dijkstra":13,lodash:28}],13:[function(require,module,exports){var _=require("lodash"),PriorityQueue=require("../data/priority-queue");module.exports=dijkstra;var DEFAULT_WEIGHT_FUNC=_.constant(1);function dijkstra(g,source,weightFn,edgeFn){return runDijkstra(g,String(source),weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runDijkstra(g,source,weightFn,edgeFn){var results={},pq=new PriorityQueue,v,vEntry;var updateNeighbors=function(edge){var w=edge.v!==v?edge.v:edge.w,wEntry=results[w],weight=weightFn(edge),distance=vEntry.distance+weight;if(weight<0){throw new Error("dijkstra does not allow negative edge weights. "+"Bad edge: "+edge+" Weight: "+weight)}if(distance0){v=pq.removeMin();vEntry=results[v];if(vEntry.distance===Number.POSITIVE_INFINITY){break}edgeFn(v).forEach(updateNeighbors)}return results}},{"../data/priority-queue":23,lodash:28}],14:[function(require,module,exports){var _=require("lodash"),tarjan=require("./tarjan");module.exports=findCycles;function findCycles(g){return _.filter(tarjan(g),function(cmpt){return cmpt.length>1})}},{"./tarjan":21,lodash:28}],15:[function(require,module,exports){var _=require("lodash");module.exports=floydWarshall;var DEFAULT_WEIGHT_FUNC=_.constant(1);function floydWarshall(g,weightFn,edgeFn){return runFloydWarshall(g,weightFn||DEFAULT_WEIGHT_FUNC,edgeFn||function(v){return g.outEdges(v)})}function runFloydWarshall(g,weightFn,edgeFn){var results={},nodes=g.nodes();nodes.forEach(function(v){results[v]={};results[v][v]={distance:0};nodes.forEach(function(w){if(v!==w){results[v][w]={distance:Number.POSITIVE_INFINITY}}});edgeFn(v).forEach(function(edge){var w=edge.v===v?edge.w:edge.v,d=weightFn(edge);results[v][w]={distance:d,predecessor:v}})});nodes.forEach(function(k){var rowK=results[k];nodes.forEach(function(i){var rowI=results[i];nodes.forEach(function(j){var ik=rowI[k];var kj=rowK[j];var ij=rowI[j];var altDistance=ik.distance+kj.distance;if(altDistance0){v=pq.removeMin();if(_.has(parents,v)){result.setEdge(v,parents[v])}else if(init){throw new Error("Input graph is not connected: "+g)}else{init=true}g.nodeEdges(v).forEach(updateNeighbors)}return result}},{"../data/priority-queue":23,"../graph":24,lodash:28}],21:[function(require,module,exports){var _=require("lodash");module.exports=tarjan;function tarjan(g){var index=0,stack=[],visited={},results=[];function dfs(v){var entry=visited[v]={onStack:true,lowlink:index,index:index++};stack.push(v);g.successors(v).forEach(function(w){if(!_.has(visited,w)){dfs(w);entry.lowlink=Math.min(entry.lowlink,visited[w].lowlink)}else if(visited[w].onStack){entry.lowlink=Math.min(entry.lowlink,visited[w].index)}});if(entry.lowlink===entry.index){var cmpt=[],w;do{w=stack.pop();visited[w].onStack=false;cmpt.push(w)}while(v!==w);results.push(cmpt)}}g.nodes().forEach(function(v){if(!_.has(visited,v)){dfs(v)}});return results}},{lodash:28}],22:[function(require,module,exports){var _=require("lodash");module.exports=topsort;topsort.CycleException=CycleException;function topsort(g){var visited={},stack={},results=[];function visit(node){if(_.has(stack,node)){throw new CycleException}if(!_.has(visited,node)){stack[node]=true;visited[node]=true;_.each(g.predecessors(node),visit);delete stack[node];results.push(node)}}_.each(g.sinks(),visit);if(_.size(visited)!==g.nodeCount()){throw new CycleException}return results}function CycleException(){}},{lodash:28}],23:[function(require,module,exports){var _=require("lodash");module.exports=PriorityQueue;function PriorityQueue(){this._arr=[];this._keyIndices={}}PriorityQueue.prototype.size=function(){return this._arr.length};PriorityQueue.prototype.keys=function(){return this._arr.map(function(x){return x.key})};PriorityQueue.prototype.has=function(key){return _.has(this._keyIndices,key)};PriorityQueue.prototype.priority=function(key){var index=this._keyIndices[key];if(index!==undefined){return this._arr[index].priority}};PriorityQueue.prototype.min=function(){if(this.size()===0){throw new Error("Queue underflow")}return this._arr[0].key};PriorityQueue.prototype.add=function(key,priority){var keyIndices=this._keyIndices;key=String(key);if(!_.has(keyIndices,key)){var arr=this._arr;var index=arr.length;keyIndices[key]=index;arr.push({key:key,priority:priority});this._decrease(index);return true}return false};PriorityQueue.prototype.removeMin=function(){this._swap(0,this._arr.length-1);var min=this._arr.pop();delete this._keyIndices[min.key];this._heapify(0);return min.key};PriorityQueue.prototype.decrease=function(key,priority){var index=this._keyIndices[key];if(priority>this._arr[index].priority){throw new Error("New priority is greater than current priority. "+"Key: "+key+" Old: "+this._arr[index].priority+" New: "+priority)}this._arr[index].priority=priority;this._decrease(index)};PriorityQueue.prototype._heapify=function(i){var arr=this._arr;var l=2*i,r=l+1,largest=i;if(l>1;if(arr[parent].priority1){this.setNode(v,value)}else{this.setNode(v)}},this);return this};Graph.prototype.setNode=function(v,value){if(_.has(this._nodes,v)){if(arguments.length>1){this._nodes[v]=value}return this}this._nodes[v]=arguments.length>1?value:this._defaultNodeLabelFn(v);if(this._isCompound){this._parent[v]=GRAPH_NODE;this._children[v]={};this._children[GRAPH_NODE][v]=true}this._in[v]={};this._preds[v]={};this._out[v]={};this._sucs[v]={};++this._nodeCount;return this};Graph.prototype.node=function(v){return this._nodes[v]};Graph.prototype.hasNode=function(v){return _.has(this._nodes,v)};Graph.prototype.removeNode=function(v){var self=this;if(_.has(this._nodes,v)){var removeEdge=function(e){self.removeEdge(self._edgeObjs[e])};delete this._nodes[v];if(this._isCompound){this._removeFromParentsChildList(v);delete this._parent[v];_.each(this.children(v),function(child){this.setParent(child)},this);delete this._children[v]}_.each(_.keys(this._in[v]),removeEdge);delete this._in[v];delete this._preds[v];_.each(_.keys(this._out[v]),removeEdge);delete this._out[v];delete this._sucs[v];--this._nodeCount}return this};Graph.prototype.setParent=function(v,parent){if(!this._isCompound){throw new Error("Cannot set parent in a non-compound graph")}if(_.isUndefined(parent)){parent=GRAPH_NODE}else{for(var ancestor=parent;!_.isUndefined(ancestor);ancestor=this.parent(ancestor)){if(ancestor===v){throw new Error("Setting "+parent+" as parent of "+v+" would create create a cycle")}}this.setNode(parent)}this.setNode(v);this._removeFromParentsChildList(v);this._parent[v]=parent;this._children[parent][v]=true;return this};Graph.prototype._removeFromParentsChildList=function(v){delete this._children[this._parent[v]][v]};Graph.prototype.parent=function(v){if(this._isCompound){var parent=this._parent[v];if(parent!==GRAPH_NODE){return parent}}};Graph.prototype.children=function(v){if(_.isUndefined(v)){v=GRAPH_NODE}if(this._isCompound){var children=this._children[v];if(children){return _.keys(children)}}else if(v===GRAPH_NODE){return this.nodes()}else if(this.hasNode(v)){return[]}};Graph.prototype.predecessors=function(v){var predsV=this._preds[v];if(predsV){return _.keys(predsV)}};Graph.prototype.successors=function(v){var sucsV=this._sucs[v];if(sucsV){return _.keys(sucsV)}};Graph.prototype.neighbors=function(v){var preds=this.predecessors(v);if(preds){return _.union(preds,this.successors(v))}};Graph.prototype.setDefaultEdgeLabel=function(newDefault){if(!_.isFunction(newDefault)){newDefault=_.constant(newDefault)}this._defaultEdgeLabelFn=newDefault;return this};Graph.prototype.edgeCount=function(){return this._edgeCount};Graph.prototype.edges=function(){return _.values(this._edgeObjs)};Graph.prototype.setPath=function(vs,value){var self=this,args=arguments;_.reduce(vs,function(v,w){if(args.length>1){self.setEdge(v,w,value)}else{self.setEdge(v,w)}return w});return this};Graph.prototype.setEdge=function(v,w,value,name){var valueSpecified=arguments.length>2;if(_.isPlainObject(arguments[0])){v=arguments[0].v;w=arguments[0].w;name=arguments[0].name;if(arguments.length===2){value=arguments[1];valueSpecified=true}}var e=edgeArgsToId(this._isDirected,v,w,name);if(_.has(this._edgeLabels,e)){if(valueSpecified){this._edgeLabels[e]=value}return this}if(!_.isUndefined(name)&&!this._isMultigraph){throw new Error("Cannot set a named edge when isMultigraph = false")}this.setNode(v);this.setNode(w);this._edgeLabels[e]=valueSpecified?value:this._defaultEdgeLabelFn(v,w,name);var edgeObj=edgeArgsToObj(this._isDirected,v,w,name);v=edgeObj.v;w=edgeObj.w;Object.freeze(edgeObj);this._edgeObjs[e]=edgeObj;incrementOrInitEntry(this._preds[w],v);incrementOrInitEntry(this._sucs[v],w);this._in[w][e]=edgeObj;this._out[v][e]=edgeObj;this._edgeCount++;return this};Graph.prototype.edge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return this._edgeLabels[e]};Graph.prototype.hasEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name);return _.has(this._edgeLabels,e)};Graph.prototype.removeEdge=function(v,w,name){var e=arguments.length===1?edgeObjToId(this._isDirected,arguments[0]):edgeArgsToId(this._isDirected,v,w,name),edge=this._edgeObjs[e];if(edge){v=edge.v;w=edge.w;delete this._edgeLabels[e];delete this._edgeObjs[e];decrementOrRemoveEntry(this._preds[w],v);decrementOrRemoveEntry(this._sucs[v],w);delete this._in[w][e];delete this._out[v][e];this._edgeCount--}return this};Graph.prototype.inEdges=function(v,u){var inV=this._in[v];if(inV){var edges=_.values(inV);if(!u){return edges}return _.filter(edges,function(edge){return edge.v===u})}};Graph.prototype.outEdges=function(v,w){var outV=this._out[v];if(outV){var edges=_.values(outV);if(!w){return edges}return _.filter(edges,function(edge){return edge.w===w})}};Graph.prototype.nodeEdges=function(v,w){var inEdges=this.inEdges(v,w);if(inEdges){return inEdges.concat(this.outEdges(v,w))}};function incrementOrInitEntry(map,k){if(_.has(map,k)){map[k]++}else{map[k]=1}}function decrementOrRemoveEntry(map,k){if(!--map[k]){delete map[k]}}function edgeArgsToId(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}return v+EDGE_KEY_DELIM+w+EDGE_KEY_DELIM+(_.isUndefined(name)?DEFAULT_EDGE_NAME:name)}function edgeArgsToObj(isDirected,v,w,name){if(!isDirected&&v>w){var tmp=v;v=w;w=tmp}var edgeObj={v:v,w:w};if(name){edgeObj.name=name}return edgeObj}function edgeObjToId(isDirected,edgeObj){return edgeArgsToId(isDirected,edgeObj.v,edgeObj.w,edgeObj.name)}},{lodash:28}],25:[function(require,module,exports){module.exports={Graph:require("./graph"),version:require("./version")}},{"./graph":24,"./version":27}],26:[function(require,module,exports){var _=require("lodash"),Graph=require("./graph");module.exports={write:write,read:read};function write(g){var json={options:{directed:g.isDirected(),multigraph:g.isMultigraph(),compound:g.isCompound()},nodes:writeNodes(g),edges:writeEdges(g)};if(!_.isUndefined(g.graph())){json.value=_.clone(g.graph())}return json}function writeNodes(g){return _.map(g.nodes(),function(v){var nodeValue=g.node(v),parent=g.parent(v),node={v:v};if(!_.isUndefined(nodeValue)){node.value=nodeValue}if(!_.isUndefined(parent)){node.parent=parent}return node})}function writeEdges(g){return _.map(g.edges(),function(e){var edgeValue=g.edge(e),edge={v:e.v,w:e.w};if(!_.isUndefined(e.name)){edge.name=e.name}if(!_.isUndefined(edgeValue)){edge.value=edgeValue}return edge})}function read(json){var g=new Graph(json.options).setGraph(json.value);_.each(json.nodes,function(entry){g.setNode(entry.v,entry.value);if(entry.parent){g.setParent(entry.v,entry.parent)}});_.each(json.edges,function(entry){g.setEdge({v:entry.v,w:entry.w,name:entry.name},entry.value)});return g}},{"./graph":24,lodash:28}],27:[function(require,module,exports){module.exports="0.8.1"},{}],28:[function(require,module,exports){(function(global){(function(){var undefined;var arrayPool=[],objectPool=[];var idCounter=0;var keyPrefix=+new Date+"";var largeArraySize=75;var maxPoolSize=40;var whitespace=" \f "+"\n\r\u2028\u2029"+" ᠎              ";var reEmptyStringLeading=/\b__p \+= '';/g,reEmptyStringMiddle=/\b(__p \+=) '' \+/g,reEmptyStringTrailing=/(__e\(.*?\)|\b__t\)) \+\n'';/g;var reEsTemplate=/\$\{([^\\}]*(?:\\.[^\\}]*)*)\}/g;var reFlags=/\w*$/;var reFuncName=/^\s*function[ \n\r\t]+\w/;var reInterpolate=/<%=([\s\S]+?)%>/g;var reLeadingSpacesAndZeros=RegExp("^["+whitespace+"]*0+(?=.$)");var reNoMatch=/($^)/;var reThis=/\bthis\b/;var reUnescapedString=/['\n\r\t\u2028\u2029\\]/g;var contextProps=["Array","Boolean","Date","Function","Math","Number","Object","RegExp","String","_","attachEvent","clearTimeout","isFinite","isNaN","parseInt","setTimeout"];var templateCounter=0;var argsClass="[object Arguments]",arrayClass="[object Array]",boolClass="[object Boolean]",dateClass="[object Date]",funcClass="[object Function]",numberClass="[object Number]",objectClass="[object Object]",regexpClass="[object RegExp]",stringClass="[object String]";var cloneableClasses={};cloneableClasses[funcClass]=false;cloneableClasses[argsClass]=cloneableClasses[arrayClass]=cloneableClasses[boolClass]=cloneableClasses[dateClass]=cloneableClasses[numberClass]=cloneableClasses[objectClass]=cloneableClasses[regexpClass]=cloneableClasses[stringClass]=true;var debounceOptions={leading:false,maxWait:0,trailing:false};var descriptor={configurable:false,enumerable:false,value:null,writable:false};var objectTypes={"boolean":false,"function":true,object:true,number:false,string:false,undefined:false};var stringEscapes={"\\":"\\","'":"'","\n":"n","\r":"r"," ":"t","\u2028":"u2028","\u2029":"u2029"};var root=objectTypes[typeof window]&&window||this;var freeExports=objectTypes[typeof exports]&&exports&&!exports.nodeType&&exports;var freeModule=objectTypes[typeof module]&&module&&!module.nodeType&&module;var moduleExports=freeModule&&freeModule.exports===freeExports&&freeExports;var freeGlobal=objectTypes[typeof global]&&global;if(freeGlobal&&(freeGlobal.global===freeGlobal||freeGlobal.window===freeGlobal)){root=freeGlobal}function baseIndexOf(array,value,fromIndex){var index=(fromIndex||0)-1,length=array?array.length:0;while(++index-1?0:-1:cache?0:-1}function cachePush(value){var cache=this.cache,type=typeof value;if(type=="boolean"||value==null){cache[value]=true}else{if(type!="number"&&type!="string"){type="object"}var key=type=="number"?value:keyPrefix+value,typeCache=cache[type]||(cache[type]={});if(type=="object"){(typeCache[key]||(typeCache[key]=[])).push(value)}else{typeCache[key]=true}}}function charAtCallback(value){return value.charCodeAt(0)}function compareAscending(a,b){var ac=a.criteria,bc=b.criteria,index=-1,length=ac.length;while(++indexother||typeof value=="undefined"){return 1}if(value/g,evaluate:/<%([\s\S]+?)%>/g,interpolate:reInterpolate,variable:"",imports:{_:lodash}};function baseBind(bindData){var func=bindData[0],partialArgs=bindData[2],thisArg=bindData[4];function bound(){if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(this instanceof bound){var thisBinding=baseCreate(func.prototype),result=func.apply(thisBinding,args||arguments);return isObject(result)?result:thisBinding}return func.apply(thisArg,args||arguments)}setBindData(bound,bindData);return bound}function baseClone(value,isDeep,callback,stackA,stackB){if(callback){var result=callback(value);if(typeof result!="undefined"){return result}}var isObj=isObject(value);if(isObj){var className=toString.call(value);if(!cloneableClasses[className]){return value}var ctor=ctorByClass[className];switch(className){case boolClass:case dateClass:return new ctor(+value);case numberClass:case stringClass:return new ctor(value);case regexpClass:result=ctor(value.source,reFlags.exec(value));result.lastIndex=value.lastIndex;return result}}else{return value}var isArr=isArray(value);if(isDeep){var initedStack=!stackA;stackA||(stackA=getArray());stackB||(stackB=getArray());var length=stackA.length;while(length--){if(stackA[length]==value){return stackB[length]}}result=isArr?ctor(value.length):{}}else{result=isArr?slice(value):assign({},value)}if(isArr){if(hasOwnProperty.call(value,"index")){result.index=value.index}if(hasOwnProperty.call(value,"input")){result.input=value.input}}if(!isDeep){return result}stackA.push(value);stackB.push(result);(isArr?forEach:forOwn)(value,function(objValue,key){result[key]=baseClone(objValue,isDeep,callback,stackA,stackB)});if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseCreate(prototype,properties){return isObject(prototype)?nativeCreate(prototype):{}}if(!nativeCreate){baseCreate=function(){function Object(){}return function(prototype){if(isObject(prototype)){Object.prototype=prototype;var result=new Object;Object.prototype=null}return result||context.Object()}}()}function baseCreateCallback(func,thisArg,argCount){if(typeof func!="function"){return identity}if(typeof thisArg=="undefined"||!("prototype"in func)){return func}var bindData=func.__bindData__;if(typeof bindData=="undefined"){if(support.funcNames){bindData=!func.name}bindData=bindData||!support.funcDecomp;if(!bindData){var source=fnToString.call(func);if(!support.funcNames){bindData=!reFuncName.test(source)}if(!bindData){bindData=reThis.test(source);setBindData(func,bindData)}}}if(bindData===false||bindData!==true&&bindData[1]&1){return func}switch(argCount){case 1:return function(value){return func.call(thisArg,value)};case 2:return function(a,b){return func.call(thisArg,a,b)};case 3:return function(value,index,collection){return func.call(thisArg,value,index,collection)};case 4:return function(accumulator,value,index,collection){return func.call(thisArg,accumulator,value,index,collection)}}return bind(func,thisArg)}function baseCreateWrapper(bindData){var func=bindData[0],bitmask=bindData[1],partialArgs=bindData[2],partialRightArgs=bindData[3],thisArg=bindData[4],arity=bindData[5];var isBind=bitmask&1,isBindKey=bitmask&2,isCurry=bitmask&4,isCurryBound=bitmask&8,key=func;function bound(){var thisBinding=isBind?thisArg:this;if(partialArgs){var args=slice(partialArgs);push.apply(args,arguments)}if(partialRightArgs||isCurry){args||(args=slice(arguments));if(partialRightArgs){push.apply(args,partialRightArgs)}if(isCurry&&args.length=largeArraySize&&indexOf===baseIndexOf,result=[];if(isLarge){var cache=createCache(values);if(cache){indexOf=cacheIndexOf;values=cache}else{isLarge=false}}while(++index-1}})}}stackA.pop();stackB.pop();if(initedStack){releaseArray(stackA);releaseArray(stackB)}return result}function baseMerge(object,source,callback,stackA,stackB){(isArray(source)?forEach:forOwn)(source,function(source,key){var found,isArr,result=source,value=object[key];if(source&&((isArr=isArray(source))||isPlainObject(source))){var stackLength=stackA.length;while(stackLength--){if(found=stackA[stackLength]==source){value=stackB[stackLength];break}}if(!found){var isShallow;if(callback){result=callback(value,source);if(isShallow=typeof result!="undefined"){value=result}}if(!isShallow){value=isArr?isArray(value)?value:[]:isPlainObject(value)?value:{}}stackA.push(source);stackB.push(value);if(!isShallow){baseMerge(value,source,callback,stackA,stackB)}}}else{if(callback){result=callback(value,source);if(typeof result=="undefined"){result=source}}if(typeof result!="undefined"){value=result}}object[key]=value})}function baseRandom(min,max){return min+floor(nativeRandom()*(max-min+1))}function baseUniq(array,isSorted,callback){var index=-1,indexOf=getIndexOf(),length=array?array.length:0,result=[];var isLarge=!isSorted&&length>=largeArraySize&&indexOf===baseIndexOf,seen=callback||isLarge?getArray():result;if(isLarge){var cache=createCache(seen);indexOf=cacheIndexOf;seen=cache}while(++index":">",'"':""","'":"'"};var htmlUnescapes=invert(htmlEscapes);var reEscapedHtml=RegExp("("+keys(htmlUnescapes).join("|")+")","g"),reUnescapedHtml=RegExp("["+keys(htmlEscapes).join("")+"]","g");var assign=function(object,source,guard){var index,iterable=object,result=iterable;if(!iterable)return result;var args=arguments,argsIndex=0,argsLength=typeof guard=="number"?2:args.length;if(argsLength>3&&typeof args[argsLength-2]=="function"){var callback=baseCreateCallback(args[--argsLength-1],args[argsLength--],2)}else if(argsLength>2&&typeof args[argsLength-1]=="function"){callback=args[--argsLength]}while(++argsIndex3&&typeof args[length-2]=="function"){var callback=baseCreateCallback(args[--length-1],args[length--],2)}else if(length>2&&typeof args[length-1]=="function"){callback=args[--length]}var sources=slice(arguments,1,length),index=-1,stackA=getArray(),stackB=getArray();while(++index-1}else if(typeof length=="number"){result=(isString(collection)?collection.indexOf(target,fromIndex):indexOf(collection,target,fromIndex))>-1}else{forOwn(collection,function(value){if(++index>=fromIndex){return!(result=value===target)}})}return result}var countBy=createAggregator(function(result,value,key){hasOwnProperty.call(result,key)?result[key]++:result[key]=1});function every(collection,callback,thisArg){var result=true;callback=lodash.createCallback(callback,thisArg,3);var index=-1,length=collection?collection.length:0;if(typeof length=="number"){while(++indexresult){result=value}}}else{callback=callback==null&&isString(collection)?charAtCallback:lodash.createCallback(callback,thisArg,3);forEach(collection,function(value,index,collection){var current=callback(value,index,collection);if(current>computed){computed=current;result=value}})}return result}function min(collection,callback,thisArg){var computed=Infinity,result=computed;if(typeof callback!="function"&&thisArg&&thisArg[callback]===collection){callback=null}if(callback==null&&isArray(collection)){var index=-1,length=collection.length;while(++index=largeArraySize&&createCache(argsIndex?args[argsIndex]:seen))}}var array=args[0],index=-1,length=array?array.length:0,result=[];outer:while(++index>>1;callback(array[mid])1?arguments:arguments[0],index=-1,length=array?max(pluck(array,"length")):0,result=Array(length<0?0:length);while(++index2?createWrapper(func,17,slice(arguments,2),null,thisArg):createWrapper(func,1,null,null,thisArg)}function bindAll(object){var funcs=arguments.length>1?baseFlatten(arguments,true,false,1):functions(object),index=-1,length=funcs.length;while(++index2?createWrapper(key,19,slice(arguments,2),null,object):createWrapper(key,3,null,null,object)}function compose(){var funcs=arguments,length=funcs.length;while(length--){if(!isFunction(funcs[length])){throw new TypeError}}return function(){var args=arguments,length=funcs.length;while(length--){args=[funcs[length].apply(this,args)]}return args[0]}}function curry(func,arity){arity=typeof arity=="number"?arity:+arity||func.length;return createWrapper(func,4,null,null,null,arity)}function debounce(func,wait,options){var args,maxTimeoutId,result,stamp,thisArg,timeoutId,trailingCall,lastCalled=0,maxWait=false,trailing=true;if(!isFunction(func)){throw new TypeError}wait=nativeMax(0,wait)||0;if(options===true){var leading=true;trailing=false}else if(isObject(options)){leading=options.leading;maxWait="maxWait"in options&&(nativeMax(wait,options.maxWait)||0);trailing="trailing"in options?options.trailing:trailing}var delayed=function(){var remaining=wait-(now()-stamp);if(remaining<=0){if(maxTimeoutId){clearTimeout(maxTimeoutId)}var isCalled=trailingCall;maxTimeoutId=timeoutId=trailingCall=undefined;if(isCalled){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}}else{timeoutId=setTimeout(delayed,remaining)}};var maxDelayed=function(){if(timeoutId){clearTimeout(timeoutId)}maxTimeoutId=timeoutId=trailingCall=undefined;if(trailing||maxWait!==wait){lastCalled=now();result=func.apply(thisArg,args);if(!timeoutId&&!maxTimeoutId){args=thisArg=null}}};return function(){args=arguments;stamp=now();thisArg=this;trailingCall=trailing&&(timeoutId||!leading);if(maxWait===false){var leadingCall=leading&&!timeoutId}else{if(!maxTimeoutId&&!leading){lastCalled=stamp}var remaining=maxWait-(stamp-lastCalled),isCalled=remaining<=0;if(isCalled){if(maxTimeoutId){maxTimeoutId=clearTimeout(maxTimeoutId)}lastCalled=stamp;result=func.apply(thisArg,args)}else if(!maxTimeoutId){maxTimeoutId=setTimeout(maxDelayed,remaining)}}if(isCalled&&timeoutId){timeoutId=clearTimeout(timeoutId)}else if(!timeoutId&&wait!==maxWait){timeoutId=setTimeout(delayed,wait)}if(leadingCall){isCalled=true;result=func.apply(thisArg,args)}if(isCalled&&!timeoutId&&!maxTimeoutId){args=thisArg=null}return result}}function defer(func){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,1);return setTimeout(function(){func.apply(undefined,args)},1)}function delay(func,wait){if(!isFunction(func)){throw new TypeError}var args=slice(arguments,2);return setTimeout(function(){func.apply(undefined,args)},wait)}function memoize(func,resolver){if(!isFunction(func)){throw new TypeError}var memoized=function(){var cache=memoized.cache,key=resolver?resolver.apply(this,arguments):keyPrefix+arguments[0];return hasOwnProperty.call(cache,key)?cache[key]:cache[key]=func.apply(this,arguments)};memoized.cache={};return memoized}function once(func){var ran,result;if(!isFunction(func)){throw new TypeError}return function(){if(ran){return result}ran=true;result=func.apply(this,arguments);func=null;return result}}function partial(func){return createWrapper(func,16,slice(arguments,1))}function partialRight(func){return createWrapper(func,32,null,slice(arguments,1))}function throttle(func,wait,options){var leading=true,trailing=true;if(!isFunction(func)){throw new TypeError}if(options===false){leading=false}else if(isObject(options)){leading="leading"in options?options.leading:leading;trailing="trailing"in options?options.trailing:trailing}debounceOptions.leading=leading;debounceOptions.maxWait=wait;debounceOptions.trailing=trailing;return debounce(func,wait,debounceOptions)}function wrap(value,wrapper){return createWrapper(wrapper,16,[value])}function constant(value){return function(){return value}}function createCallback(func,thisArg,argCount){var type=typeof func;if(func==null||type=="function"){return baseCreateCallback(func,thisArg,argCount)}if(type!="object"){return property(func)}var props=keys(func),key=props[0],a=func[key];if(props.length==1&&a===a&&!isObject(a)){return function(object){var b=object[key];return a===b&&(a!==0||1/a==1/b)}}return function(object){var length=props.length,result=false;while(length--){if(!(result=baseIsEqual(object[props[length]],func[props[length]],null,true))){break}}return result}}function escape(string){return string==null?"":String(string).replace(reUnescapedHtml,escapeHtmlChar)}function identity(value){return value}function mixin(object,source,options){var chain=true,methodNames=source&&functions(source);if(!source||!options&&!methodNames.length){if(options==null){options=source}ctor=lodashWrapper;source=object;object=lodash;methodNames=functions(source)}if(options===false){chain=false}else if(isObject(options)&&"chain"in options){chain=options.chain}var ctor=object,isFunc=isFunction(ctor);forEach(methodNames,function(methodName){var func=object[methodName]=source[methodName];if(isFunc){ctor.prototype[methodName]=function(){var chainAll=this.__chain__,value=this.__wrapped__,args=[value];push.apply(args,arguments);var result=func.apply(object,args);if(chain||chainAll){if(value===result&&isObject(result)){return this}result=new ctor(result);result.__chain__=chainAll}return result}}})}function noConflict(){context._=oldDash;return this}function noop(){}var now=isNative(now=Date.now)&&now||function(){return(new Date).getTime()};var parseInt=nativeParseInt(whitespace+"08")==8?nativeParseInt:function(value,radix){return nativeParseInt(isString(value)?value.replace(reLeadingSpacesAndZeros,""):value,radix||0)};function property(key){return function(object){return object[key]}}function random(min,max,floating){var noMin=min==null,noMax=max==null;if(floating==null){if(typeof min=="boolean"&&noMax){floating=min;min=1}else if(!noMax&&typeof max=="boolean"){floating=max;noMax=true}}if(noMin&&noMax){max=1}min=+min||0;if(noMax){max=min;min=0}else{max=+max||0}if(floating||min%1||max%1){var rand=nativeRandom();return nativeMin(min+rand*(max-min+parseFloat("1e-"+((rand+"").length-1))),max)}return baseRandom(min,max)}function result(object,key){if(object){var value=object[key];return isFunction(value)?object[key]():value}}function template(text,data,options){var settings=lodash.templateSettings;text=String(text||"");options=defaults({},options,settings);var imports=defaults({},options.imports,settings.imports),importsKeys=keys(imports),importsValues=values(imports);var isEvaluating,index=0,interpolate=options.interpolate||reNoMatch,source="__p += '";var reDelimiters=RegExp((options.escape||reNoMatch).source+"|"+interpolate.source+"|"+(interpolate===reInterpolate?reEsTemplate:reNoMatch).source+"|"+(options.evaluate||reNoMatch).source+"|$","g");text.replace(reDelimiters,function(match,escapeValue,interpolateValue,esTemplateValue,evaluateValue,offset){interpolateValue||(interpolateValue=esTemplateValue);source+=text.slice(index,offset).replace(reUnescapedString,escapeStringChar);if(escapeValue){source+="' +\n__e("+escapeValue+") +\n'"}if(evaluateValue){isEvaluating=true;source+="';\n"+evaluateValue+";\n__p += '"}if(interpolateValue){source+="' +\n((__t = ("+interpolateValue+")) == null ? '' : __t) +\n'"}index=offset+match.length;return match});source+="';\n";var variable=options.variable,hasVariable=variable;if(!hasVariable){variable="obj";source="with ("+variable+") {\n"+source+"\n}\n"}source=(isEvaluating?source.replace(reEmptyStringLeading,""):source).replace(reEmptyStringMiddle,"$1").replace(reEmptyStringTrailing,"$1;");source="function("+variable+") {\n"+(hasVariable?"":variable+" || ("+variable+" = {});\n")+"var __t, __p = '', __e = _.escape"+(isEvaluating?", __j = Array.prototype.join;\n"+"function print() { __p += __j.call(arguments, '') }\n":";\n")+source+"return __p\n}";var sourceURL="\n/*\n//# sourceURL="+(options.sourceURL||"/lodash/template/source["+templateCounter++ +"]")+"\n*/";try{var result=Function(importsKeys,"return "+source+sourceURL).apply(undefined,importsValues)}catch(e){e.source=source;throw e}if(data){return result(data)}result.source=source;return result}function times(n,callback,thisArg){n=(n=+n)>-1?n:0;var index=-1,result=Array(n);callback=baseCreateCallback(callback,thisArg,1);while(++indexa?this[a+this.length]:this[a]:d.call(this)},pushStack:function(a){var b=m.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a,b){return m.each(this,a,b)},map:function(a){return this.pushStack(m.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},m.extend=m.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||m.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&&(m.isPlainObject(c)||(b=m.isArray(c)))?(b?(b=!1,f=a&&m.isArray(a)?a:[]):f=a&&m.isPlainObject(a)?a:{},g[d]=m.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},m.extend({expando:"jQuery"+(l+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===m.type(a)},isArray:Array.isArray||function(a){return"array"===m.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){return!m.isArray(a)&&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"!==m.type(a)||a.nodeType||m.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(k.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&&m.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(o,"ms-").replace(p,q)},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=r(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:function(a){return null==a?"":(a+"").replace(n,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(r(Object(a))?m.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=r(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),m.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||m.guid++,e):void 0},now:function(){return+new Date},support:k}),m.each("Boolean Number String Function Array Date RegExp Object Error".split(" "),function(a,b){h["[object "+b+"]"]=b.toLowerCase()});function r(a){var b=a.length,c=m.type(a);return"function"===c||m.isWindow(a)?!1:1===a.nodeType&&b?!0:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var s=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+-new Date,v=a.document,w=0,x=0,y=gb(),z=gb(),A=gb(),B=function(a,b){return a===b&&(l=!0),0},C="undefined",D=1<<31,E={}.hasOwnProperty,F=[],G=F.pop,H=F.push,I=F.push,J=F.slice,K=F.indexOf||function(a){for(var b=0,c=this.length;c>b;b++)if(this[b]===a)return b;return-1},L="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",M="[\\x20\\t\\r\\n\\f]",N="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",O=N.replace("w","w#"),P="\\["+M+"*("+N+")(?:"+M+"*([*^$|!~]?=)"+M+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+O+"))|)"+M+"*\\]",Q=":("+N+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+P+")*)|.*)\\)|)",R=new RegExp("^"+M+"+|((?:^|[^\\\\])(?:\\\\.)*)"+M+"+$","g"),S=new RegExp("^"+M+"*,"+M+"*"),T=new RegExp("^"+M+"*([>+~]|"+M+")"+M+"*"),U=new RegExp("="+M+"*([^\\]'\"]*?)"+M+"*\\]","g"),V=new RegExp(Q),W=new RegExp("^"+O+"$"),X={ID:new RegExp("^#("+N+")"),CLASS:new RegExp("^\\.("+N+")"),TAG:new RegExp("^("+N.replace("w","w*")+")"),ATTR:new RegExp("^"+P),PSEUDO:new RegExp("^"+Q),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+L+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/^(?:input|select|textarea|button)$/i,Z=/^h\d$/i,$=/^[^{]+\{\s*\[native \w/,_=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ab=/[+~]/,bb=/'|\\/g,cb=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),db=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{I.apply(F=J.call(v.childNodes),v.childNodes),F[v.childNodes.length].nodeType}catch(eb){I={apply:F.length?function(a,b){H.apply(a,J.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fb(a,b,d,e){var f,h,j,k,l,o,r,s,w,x;if((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,d=d||[],!a||"string"!=typeof a)return d;if(1!==(k=b.nodeType)&&9!==k)return[];if(p&&!e){if(f=_.exec(a))if(j=f[1]){if(9===k){if(h=b.getElementById(j),!h||!h.parentNode)return d;if(h.id===j)return d.push(h),d}else if(b.ownerDocument&&(h=b.ownerDocument.getElementById(j))&&t(b,h)&&h.id===j)return d.push(h),d}else{if(f[2])return I.apply(d,b.getElementsByTagName(a)),d;if((j=f[3])&&c.getElementsByClassName&&b.getElementsByClassName)return I.apply(d,b.getElementsByClassName(j)),d}if(c.qsa&&(!q||!q.test(a))){if(s=r=u,w=b,x=9===k&&a,1===k&&"object"!==b.nodeName.toLowerCase()){o=g(a),(r=b.getAttribute("id"))?s=r.replace(bb,"\\$&"):b.setAttribute("id",s),s="[id='"+s+"'] ",l=o.length;while(l--)o[l]=s+qb(o[l]);w=ab.test(a)&&ob(b.parentNode)||b,x=o.join(",")}if(x)try{return I.apply(d,w.querySelectorAll(x)),d}catch(y){}finally{r||b.removeAttribute("id")}}}return i(a.replace(R,"$1"),b,d,e)}function gb(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function hb(a){return a[u]=!0,a}function ib(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function jb(a,b){var c=a.split("|"),e=a.length;while(e--)d.attrHandle[c[e]]=b}function kb(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||D)-(~a.sourceIndex||D);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function lb(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function mb(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function nb(a){return hb(function(b){return b=+b,hb(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 ob(a){return a&&typeof a.getElementsByTagName!==C&&a}c=fb.support={},f=fb.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fb.setDocument=function(a){var b,e=a?a.ownerDocument||a:v,g=e.defaultView;return e!==n&&9===e.nodeType&&e.documentElement?(n=e,o=e.documentElement,p=!f(e),g&&g!==g.top&&(g.addEventListener?g.addEventListener("unload",function(){m()},!1):g.attachEvent&&g.attachEvent("onunload",function(){m()})),c.attributes=ib(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ib(function(a){return a.appendChild(e.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=$.test(e.getElementsByClassName)&&ib(function(a){return a.innerHTML="
",a.firstChild.className="i",2===a.getElementsByClassName("i").length}),c.getById=ib(function(a){return o.appendChild(a).id=u,!e.getElementsByName||!e.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if(typeof b.getElementById!==C&&p){var c=b.getElementById(a);return c&&c.parentNode?[c]:[]}},d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(cb,db);return function(a){var c=typeof a.getAttributeNode!==C&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return typeof b.getElementsByTagName!==C?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!==C&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=$.test(e.querySelectorAll))&&(ib(function(a){a.innerHTML="",a.querySelectorAll("[msallowclip^='']").length&&q.push("[*^$]="+M+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+M+"*(?:value|"+L+")"),a.querySelectorAll(":checked").length||q.push(":checked")}),ib(function(a){var b=e.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+M+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=$.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ib(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",Q)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=$.test(o.compareDocumentPosition),t=b||$.test(o.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},B=b?function(a,b){if(a===b)return l=!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===v&&t(v,a)?-1:b===e||b.ownerDocument===v&&t(v,b)?1:k?K.call(k,a)-K.call(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,f=a.parentNode,g=b.parentNode,h=[a],i=[b];if(!f||!g)return a===e?-1:b===e?1:f?-1:g?1:k?K.call(k,a)-K.call(k,b):0;if(f===g)return kb(a,b);c=a;while(c=c.parentNode)h.unshift(c);c=b;while(c=c.parentNode)i.unshift(c);while(h[d]===i[d])d++;return d?kb(h[d],i[d]):h[d]===v?-1:i[d]===v?1:0},e):n},fb.matches=function(a,b){return fb(a,null,null,b)},fb.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(U,"='$1']"),!(!c.matchesSelector||!p||r&&r.test(b)||q&&q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fb(b,n,null,[a]).length>0},fb.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fb.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&E.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fb.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fb.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fb.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=fb.selectors={cacheLength:50,createPseudo:hb,match:X,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(cb,db),a[3]=(a[3]||a[4]||a[5]||"").replace(cb,db),"~="===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]||fb.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]&&fb.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return X.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&V.test(c)&&(b=g(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(cb,db).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+M+")"+a+"("+M+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||typeof a.getAttribute!==C&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fb.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(),s=!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&&s){k=q[u]||(q[u]={}),j=k[a]||[],n=j[0]===w&&j[1],m=j[0]===w&&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]=[w,n,m];break}}else if(s&&(j=(b[u]||(b[u]={}))[a])&&j[0]===w)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&&(s&&((l[u]||(l[u]={}))[a]=[w,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()]||fb.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?hb(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=K.call(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:hb(function(a){var b=[],c=[],d=h(a.replace(R,"$1"));return d[u]?hb(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:hb(function(a){return function(b){return fb(a,b).length>0}}),contains:hb(function(a){return function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:hb(function(a){return W.test(a||"")||fb.error("unsupported lang: "+a),a=a.replace(cb,db).toLowerCase(),function(b){var c;do if(c=p?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===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.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 Z.test(a.nodeName)},input:function(a){return Y.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:nb(function(){return[0]}),last:nb(function(a,b){return[b-1]}),eq:nb(function(a,b,c){return[0>c?c+b:c]}),even:nb(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:nb(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:nb(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:nb(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function rb(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;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=[w,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[u]||(b[u]={}),(h=i[d])&&h[0]===w&&h[1]===f)return j[2]=h[2];if(i[d]=j,j[2]=a(b,c,g))return!0}}}function sb(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 tb(a,b,c){for(var d=0,e=b.length;e>d;d++)fb(a,b[d],c);return c}function ub(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 vb(a,b,c,d,e,f){return d&&!d[u]&&(d=vb(d)),e&&!e[u]&&(e=vb(e,f)),hb(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||tb(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ub(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ub(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?K.call(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ub(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):I.apply(g,r)})}function wb(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=rb(function(a){return a===b},h,!0),l=rb(function(a){return K.call(b,a)>-1},h,!0),m=[function(a,c,d){return!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d))}];f>i;i++)if(c=d.relative[a[i].type])m=[rb(sb(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return vb(i>1&&sb(m),i>1&&qb(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(R,"$1"),c,e>i&&wb(a.slice(i,e)),f>e&&wb(a=a.slice(e)),f>e&&qb(a))}m.push(c)}return sb(m)}function xb(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,m,o,p=0,q="0",r=f&&[],s=[],t=j,u=f||e&&d.find.TAG("*",k),v=w+=null==t?1:Math.random()||.1,x=u.length;for(k&&(j=g!==n&&g);q!==x&&null!=(l=u[q]);q++){if(e&&l){m=0;while(o=a[m++])if(o(l,g,h)){i.push(l);break}k&&(w=v)}c&&((l=!o&&l)&&p--,f&&r.push(l))}if(p+=q,c&&q!==p){m=0;while(o=b[m++])o(r,s,g,h);if(f){if(p>0)while(q--)r[q]||s[q]||(s[q]=G.call(i));s=ub(s)}I.apply(i,s),k&&!f&&s.length>0&&p+b.length>1&&fb.uniqueSort(i)}return k&&(w=v,j=t),r};return c?hb(f):f}return h=fb.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wb(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xb(e,d)),f.selector=a}return f},i=fb.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(cb,db),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=X.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(cb,db),ab.test(j[0].type)&&ob(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qb(j),!a)return I.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,ab.test(a)&&ob(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ib(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ib(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||jb("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ib(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||jb("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ib(function(a){return null==a.getAttribute("disabled")})||jb(L,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fb}(a);m.find=s,m.expr=s.selectors,m.expr[":"]=m.expr.pseudos,m.unique=s.uniqueSort,m.text=s.getText,m.isXMLDoc=s.isXML,m.contains=s.contains;var t=m.expr.match.needsContext,u=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,v=/^.[^:#\[\.,]*$/;function w(a,b,c){if(m.isFunction(b))return m.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return m.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(v.test(b))return m.filter(b,a,c);b=m.filter(b,a)}return m.grep(a,function(a){return m.inArray(a,b)>=0!==c})}m.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?m.find.matchesSelector(d,a)?[d]:[]:m.find.matches(a,m.grep(b,function(a){return 1===a.nodeType}))},m.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(m(a).filter(function(){for(b=0;e>b;b++)if(m.contains(d[b],this))return!0}));for(b=0;e>b;b++)m.find(a,d[b],c);return c=this.pushStack(e>1?m.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(w(this,a||[],!1))},not:function(a){return this.pushStack(w(this,a||[],!0))},is:function(a){return!!w(this,"string"==typeof a&&t.test(a)?m(a):a||[],!1).length}});var x,y=a.document,z=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,A=m.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]:z.exec(a),!c||!c[1]&&b)return!b||b.jquery?(b||x).find(a):this.constructor(b).find(a);if(c[1]){if(b=b instanceof m?b[0]:b,m.merge(this,m.parseHTML(c[1],b&&b.nodeType?b.ownerDocument||b:y,!0)),u.test(c[1])&&m.isPlainObject(b))for(c in b)m.isFunction(this[c])?this[c](b[c]):this.attr(c,b[c]);return this}if(d=y.getElementById(c[2]),d&&d.parentNode){if(d.id!==c[2])return x.find(a);this.length=1,this[0]=d}return this.context=y,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):m.isFunction(a)?"undefined"!=typeof x.ready?x.ready(a):a(m):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),m.makeArray(a,this))};A.prototype=m.fn,x=m(y);var B=/^(?:parents|prev(?:Until|All))/,C={children:!0,contents:!0,next:!0,prev:!0};m.extend({dir:function(a,b,c){var d=[],e=a[b];while(e&&9!==e.nodeType&&(void 0===c||1!==e.nodeType||!m(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}}),m.fn.extend({has:function(a){var b,c=m(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(m.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=t.test(a)||"string"!=typeof a?m(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&&m.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?m.unique(f):f)},index:function(a){return a?"string"==typeof a?m.inArray(this[0],m(a)):m.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(m.unique(m.merge(this.get(),m(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function D(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}m.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return m.dir(a,"parentNode")},parentsUntil:function(a,b,c){return m.dir(a,"parentNode",c)},next:function(a){return D(a,"nextSibling")},prev:function(a){return D(a,"previousSibling")},nextAll:function(a){return m.dir(a,"nextSibling")},prevAll:function(a){return m.dir(a,"previousSibling")},nextUntil:function(a,b,c){return m.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return m.dir(a,"previousSibling",c)},siblings:function(a){return m.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return m.sibling(a.firstChild)},contents:function(a){return m.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:m.merge([],a.childNodes)}},function(a,b){m.fn[a]=function(c,d){var e=m.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=m.filter(d,e)),this.length>1&&(C[a]||(e=m.unique(e)),B.test(a)&&(e=e.reverse())),this.pushStack(e)}});var E=/\S+/g,F={};function G(a){var b=F[a]={};return m.each(a.match(E)||[],function(a,c){b[c]=!0}),b}m.Callbacks=function(a){a="string"==typeof a?F[a]||G(a):m.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){m.each(b,function(b,c){var d=m.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&&m.each(arguments,function(a,c){var d;while((d=m.inArray(c,h,d))>-1)h.splice(d,1),b&&(e>=d&&e--,f>=d&&f--)}),this},has:function(a){return a?m.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},m.extend({Deferred:function(a){var b=[["resolve","done",m.Callbacks("once memory"),"resolved"],["reject","fail",m.Callbacks("once memory"),"rejected"],["notify","progress",m.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return m.Deferred(function(c){m.each(b,function(b,f){var g=m.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&m.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?m.extend(a,d):d}},e={};return d.pipe=d.then,m.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&&m.isFunction(a.promise)?e:0,g=1===f?a:m.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]&&m.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 H;m.fn.ready=function(a){return m.ready.promise().done(a),this},m.extend({isReady:!1,readyWait:1,holdReady:function(a){a?m.readyWait++:m.ready(!0)},ready:function(a){if(a===!0?!--m.readyWait:!m.isReady){if(!y.body)return setTimeout(m.ready);m.isReady=!0,a!==!0&&--m.readyWait>0||(H.resolveWith(y,[m]),m.fn.triggerHandler&&(m(y).triggerHandler("ready"),m(y).off("ready")))}}});function I(){y.addEventListener?(y.removeEventListener("DOMContentLoaded",J,!1),a.removeEventListener("load",J,!1)):(y.detachEvent("onreadystatechange",J),a.detachEvent("onload",J))}function J(){(y.addEventListener||"load"===event.type||"complete"===y.readyState)&&(I(),m.ready())}m.ready.promise=function(b){if(!H)if(H=m.Deferred(),"complete"===y.readyState)setTimeout(m.ready);else if(y.addEventListener)y.addEventListener("DOMContentLoaded",J,!1),a.addEventListener("load",J,!1);else{y.attachEvent("onreadystatechange",J),a.attachEvent("onload",J);var c=!1;try{c=null==a.frameElement&&y.documentElement}catch(d){}c&&c.doScroll&&!function e(){if(!m.isReady){try{c.doScroll("left")}catch(a){return setTimeout(e,50)}I(),m.ready()}}()}return H.promise(b)};var K="undefined",L;for(L in m(k))break;k.ownLast="0"!==L,k.inlineBlockNeedsLayout=!1,m(function(){var a,b,c,d;c=y.getElementsByTagName("body")[0],c&&c.style&&(b=y.createElement("div"),d=y.createElement("div"),d.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(d).appendChild(b),typeof b.style.zoom!==K&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",k.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(d))}),function(){var a=y.createElement("div");if(null==k.deleteExpando){k.deleteExpando=!0;try{delete a.test}catch(b){k.deleteExpando=!1}}a=null}(),m.acceptData=function(a){var b=m.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b};var M=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,N=/([A-Z])/g;function O(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(N,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:M.test(c)?m.parseJSON(c):c}catch(e){}m.data(a,b,c)}else c=void 0}return c}function P(a){var b;for(b in a)if(("data"!==b||!m.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function Q(a,b,d,e){if(m.acceptData(a)){var f,g,h=m.expando,i=a.nodeType,j=i?m.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()||m.guid++:h),j[k]||(j[k]=i?{}:{toJSON:m.noop}),("object"==typeof b||"function"==typeof b)&&(e?j[k]=m.extend(j[k],b):j[k].data=m.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[m.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[m.camelCase(b)])):f=g,f}}function R(a,b,c){if(m.acceptData(a)){var d,e,f=a.nodeType,g=f?m.cache:a,h=f?a[m.expando]:m.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){m.isArray(b)?b=b.concat(m.map(b,m.camelCase)):b in d?b=[b]:(b=m.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!P(d):!m.isEmptyObject(d))return}(c||(delete g[h].data,P(g[h])))&&(f?m.cleanData([a],!0):k.deleteExpando||g!=g.window?delete g[h]:g[h]=null)}}}m.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?m.cache[a[m.expando]]:a[m.expando],!!a&&!P(a)},data:function(a,b,c){return Q(a,b,c)},removeData:function(a,b){return R(a,b)},_data:function(a,b,c){return Q(a,b,c,!0)},_removeData:function(a,b){return R(a,b,!0)}}),m.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=m.data(f),1===f.nodeType&&!m._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=m.camelCase(d.slice(5)),O(f,d,e[d])));m._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){m.data(this,a)}):arguments.length>1?this.each(function(){m.data(this,a,b)}):f?O(f,a,m.data(f,a)):void 0},removeData:function(a){return this.each(function(){m.removeData(this,a)})}}),m.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=m._data(a,b),c&&(!d||m.isArray(c)?d=m._data(a,b,m.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=m.queue(a,b),d=c.length,e=c.shift(),f=m._queueHooks(a,b),g=function(){m.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 m._data(a,c)||m._data(a,c,{empty:m.Callbacks("once memory").add(function(){m._removeData(a,b+"queue"),m._removeData(a,c)})})}}),m.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},W=/^(?:checkbox|radio)$/i;!function(){var a=y.createElement("input"),b=y.createElement("div"),c=y.createDocumentFragment();if(b.innerHTML="
a",k.leadingWhitespace=3===b.firstChild.nodeType,k.tbody=!b.getElementsByTagName("tbody").length,k.htmlSerialize=!!b.getElementsByTagName("link").length,k.html5Clone="<:nav>"!==y.createElement("nav").cloneNode(!0).outerHTML,a.type="checkbox",a.checked=!0,c.appendChild(a),k.appendChecked=a.checked,b.innerHTML="",k.noCloneChecked=!!b.cloneNode(!0).lastChild.defaultValue,c.appendChild(b),b.innerHTML="",k.checkClone=b.cloneNode(!0).cloneNode(!0).lastChild.checked,k.noCloneEvent=!0,b.attachEvent&&(b.attachEvent("onclick",function(){k.noCloneEvent=!1}),b.cloneNode(!0).click()),null==k.deleteExpando){k.deleteExpando=!0;try{delete b.test}catch(d){k.deleteExpando=!1}}}(),function(){var b,c,d=y.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(k[b+"Bubbles"]=c in a)||(d.setAttribute(c,"t"),k[b+"Bubbles"]=d.attributes[c].expando===!1);d=null}();var X=/^(?:input|select|textarea)$/i,Y=/^key/,Z=/^(?:mouse|pointer|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=/^([^.]*)(?:\.(.+)|)$/;function ab(){return!0}function bb(){return!1}function cb(){try{return y.activeElement}catch(a){}}m.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=m.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return typeof m===K||a&&m.event.triggered===a.type?void 0:m.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(E)||[""],h=b.length;while(h--)f=_.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=m.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=m.event.special[o]||{},l=m.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&m.expr.match.needsContext.test(e),namespace:p.join(".")},i),(n=g[o])||(n=g[o]=[],n.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?n.splice(n.delegateCount++,0,l):n.push(l),m.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,n,o,p,q,r=m.hasData(a)&&m._data(a);if(r&&(k=r.events)){b=(b||"").match(E)||[""],j=b.length;while(j--)if(h=_.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=m.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,n=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=n.length;while(f--)g=n[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(n.splice(f,1),g.selector&&n.delegateCount--,l.remove&&l.remove.call(a,g));i&&!n.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||m.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)m.event.remove(a,o+b[j],c,d,!0);m.isEmptyObject(k)&&(delete r.handle,m._removeData(a,"events"))}},trigger:function(b,c,d,e){var f,g,h,i,k,l,n,o=[d||y],p=j.call(b,"type")?b.type:b,q=j.call(b,"namespace")?b.namespace.split("."):[];if(h=l=d=d||y,3!==d.nodeType&&8!==d.nodeType&&!$.test(p+m.event.triggered)&&(p.indexOf(".")>=0&&(q=p.split("."),p=q.shift(),q.sort()),g=p.indexOf(":")<0&&"on"+p,b=b[m.expando]?b:new m.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]:m.makeArray(c,[b]),k=m.event.special[p]||{},e||!k.trigger||k.trigger.apply(d,c)!==!1)){if(!e&&!k.noBubble&&!m.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||y)&&o.push(l.defaultView||l.parentWindow||a)}n=0;while((h=o[n++])&&!b.isPropagationStopped())b.type=n>1?i:k.bindType||p,f=(m._data(h,"events")||{})[b.type]&&m._data(h,"handle"),f&&f.apply(h,c),f=g&&h[g],f&&f.apply&&m.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)&&m.acceptData(d)&&g&&d[p]&&!m.isWindow(d)){l=d[g],l&&(d[g]=null),m.event.triggered=p;try{d[p]()}catch(r){}m.event.triggered=void 0,l&&(d[g]=l)}return b.result}},dispatch:function(a){a=m.event.fix(a);var b,c,e,f,g,h=[],i=d.call(arguments),j=(m._data(this,"events")||{})[a.type]||[],k=m.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=m.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=((m.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?m(c,this).index(i)>=0:m.find(c,this,null,[i]).length),e[c]&&e.push(d);e.length&&g.push({elem:i,handlers:e})}return h]","i"),hb=/^\s+/,ib=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,jb=/<([\w:]+)/,kb=/\s*$/g,rb={option:[1,""],legend:[1,"
","
"],area:[1,"",""],param:[1,"",""],thead:[1,"","
"],tr:[2,"","
"],col:[2,"","
"],td:[3,"","
"],_default:k.htmlSerialize?[0,"",""]:[1,"X
","
"]},sb=db(y),tb=sb.appendChild(y.createElement("div"));rb.optgroup=rb.option,rb.tbody=rb.tfoot=rb.colgroup=rb.caption=rb.thead,rb.th=rb.td;function ub(a,b){var c,d,e=0,f=typeof a.getElementsByTagName!==K?a.getElementsByTagName(b||"*"):typeof a.querySelectorAll!==K?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||m.nodeName(d,b)?f.push(d):m.merge(f,ub(d,b));return void 0===b||b&&m.nodeName(a,b)?m.merge([a],f):f}function vb(a){W.test(a.type)&&(a.defaultChecked=a.checked)}function wb(a,b){return m.nodeName(a,"table")&&m.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function xb(a){return a.type=(null!==m.find.attr(a,"type"))+"/"+a.type,a}function yb(a){var b=pb.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function zb(a,b){for(var c,d=0;null!=(c=a[d]);d++)m._data(c,"globalEval",!b||m._data(b[d],"globalEval"))}function Ab(a,b){if(1===b.nodeType&&m.hasData(a)){var c,d,e,f=m._data(a),g=m._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++)m.event.add(b,c,h[c][d])}g.data&&(g.data=m.extend({},g.data))}}function Bb(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!k.noCloneEvent&&b[m.expando]){e=m._data(b);for(d in e.events)m.removeEvent(b,d,e.handle);b.removeAttribute(m.expando)}"script"===c&&b.text!==a.text?(xb(b).text=a.text,yb(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),k.html5Clone&&a.innerHTML&&!m.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&W.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)}}m.extend({clone:function(a,b,c){var d,e,f,g,h,i=m.contains(a.ownerDocument,a);if(k.html5Clone||m.isXMLDoc(a)||!gb.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(tb.innerHTML=a.outerHTML,tb.removeChild(f=tb.firstChild)),!(k.noCloneEvent&&k.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||m.isXMLDoc(a)))for(d=ub(f),h=ub(a),g=0;null!=(e=h[g]);++g)d[g]&&Bb(e,d[g]);if(b)if(c)for(h=h||ub(a),d=d||ub(f),g=0;null!=(e=h[g]);g++)Ab(e,d[g]);else Ab(a,f);return d=ub(f,"script"),d.length>0&&zb(d,!i&&ub(a,"script")),d=h=e=null,f},buildFragment:function(a,b,c,d){for(var e,f,g,h,i,j,l,n=a.length,o=db(b),p=[],q=0;n>q;q++)if(f=a[q],f||0===f)if("object"===m.type(f))m.merge(p,f.nodeType?[f]:f);else if(lb.test(f)){h=h||o.appendChild(b.createElement("div")),i=(jb.exec(f)||["",""])[1].toLowerCase(),l=rb[i]||rb._default,h.innerHTML=l[1]+f.replace(ib,"<$1>")+l[2],e=l[0];while(e--)h=h.lastChild;if(!k.leadingWhitespace&&hb.test(f)&&p.push(b.createTextNode(hb.exec(f)[0])),!k.tbody){f="table"!==i||kb.test(f)?""!==l[1]||kb.test(f)?0:h:h.firstChild,e=f&&f.childNodes.length;while(e--)m.nodeName(j=f.childNodes[e],"tbody")&&!j.childNodes.length&&f.removeChild(j)}m.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),k.appendChecked||m.grep(ub(p,"input"),vb),q=0;while(f=p[q++])if((!d||-1===m.inArray(f,d))&&(g=m.contains(f.ownerDocument,f),h=ub(o.appendChild(f),"script"),g&&zb(h),c)){e=0;while(f=h[e++])ob.test(f.type||"")&&c.push(f)}return h=null,o},cleanData:function(a,b){for(var d,e,f,g,h=0,i=m.expando,j=m.cache,l=k.deleteExpando,n=m.event.special;null!=(d=a[h]);h++)if((b||m.acceptData(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)n[e]?m.event.remove(d,e):m.removeEvent(d,e,g.handle);j[f]&&(delete j[f],l?delete d[i]:typeof d.removeAttribute!==K?d.removeAttribute(i):d[i]=null,c.push(f))}}}),m.fn.extend({text:function(a){return V(this,function(a){return void 0===a?m.text(this):this.empty().append((this[0]&&this[0].ownerDocument||y).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=wb(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=wb(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?m.filter(a,this):this,e=0;null!=(c=d[e]);e++)b||1!==c.nodeType||m.cleanData(ub(c)),c.parentNode&&(b&&m.contains(c.ownerDocument,c)&&zb(ub(c,"script")),c.parentNode.removeChild(c));return this},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&m.cleanData(ub(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&m.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 m.clone(this,a,b)})},html:function(a){return V(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(fb,""):void 0;if(!("string"!=typeof a||mb.test(a)||!k.htmlSerialize&&gb.test(a)||!k.leadingWhitespace&&hb.test(a)||rb[(jb.exec(a)||["",""])[1].toLowerCase()])){a=a.replace(ib,"<$1>");try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(m.cleanData(ub(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,m.cleanData(ub(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,l=this.length,n=this,o=l-1,p=a[0],q=m.isFunction(p);if(q||l>1&&"string"==typeof p&&!k.checkClone&&nb.test(p))return this.each(function(c){var d=n.eq(c);q&&(a[0]=p.call(this,c,d.html())),d.domManip(a,b)});if(l&&(i=m.buildFragment(a,this[0].ownerDocument,!1,this),c=i.firstChild,1===i.childNodes.length&&(i=c),c)){for(g=m.map(ub(i,"script"),xb),f=g.length;l>j;j++)d=i,j!==o&&(d=m.clone(d,!0,!0),f&&m.merge(g,ub(d,"script"))),b.call(this[j],d,j);if(f)for(h=g[g.length-1].ownerDocument,m.map(g,yb),j=0;f>j;j++)d=g[j],ob.test(d.type||"")&&!m._data(d,"globalEval")&&m.contains(h,d)&&(d.src?m._evalUrl&&m._evalUrl(d.src):m.globalEval((d.text||d.textContent||d.innerHTML||"").replace(qb,"")));i=c=null}return this}}),m.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){m.fn[a]=function(a){for(var c,d=0,e=[],g=m(a),h=g.length-1;h>=d;d++)c=d===h?this:this.clone(!0),m(g[d])[b](c),f.apply(e,c.get());return this.pushStack(e)}});var Cb,Db={};function Eb(b,c){var d,e=m(c.createElement(b)).appendTo(c.body),f=a.getDefaultComputedStyle&&(d=a.getDefaultComputedStyle(e[0]))?d.display:m.css(e[0],"display");return e.detach(),f}function Fb(a){var b=y,c=Db[a];return c||(c=Eb(a,b),"none"!==c&&c||(Cb=(Cb||m("