You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by davies <gi...@git.apache.org> on 2014/07/17 10:17:19 UTC

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

GitHub user davies opened a pull request:

    https://github.com/apache/spark/pull/1460

    [SPARK-2538] [PySpark] Hash based disk spilling aggregation

    During aggregation in Python worker, if the memory usage is above spark.executor.memory, it will do disk spilling aggregation. 
    
    It will split the aggregation into multiple stage, in each stage, it will partition the aggregated data by hash and dump them into disks. After all the data are aggregated, it will merge all the stages together (partition by partition).

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/davies/spark spill

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/1460.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1460
    
----
commit f933713ed628779309fab0da76045f8750d6b350
Author: Davies Liu <da...@gmail.com>
Date:   2014-07-17T08:03:32Z

    Hash based disk spilling aggregation

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266513
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    --- End diff --
    
    Some people is  trying to make PySpark follow PEP8 with line width 80, we will have a pep8 checker for ensure this later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49671323
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16929/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207645
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    --- End diff --
    
    Have you looked into what it would take to support Windows? I guess right now it will keep returning 0 there and never spill?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207614
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1247,15 +1262,16 @@ def combineLocally(iterator):
                 return combiners.iteritems()
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    +        serializer = self.ctx.serializer
    +        spill = ((self.ctx._conf.get("spark.shuffle.spill") or 'True').lower()
    +                in ('true', '1', 'yes'))
    --- End diff --
    
    Probably just want to allow "true" to be similar to the way it works in Java and Scala. So just check `self.ctx._conf.get("spark.shuffle.spill", "true") == "true"`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50119786
  
    Awesome!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49930734
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17053/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15211390
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    --- End diff --
    
    Yes, this external merger will not support Windows. Because this is an improvement feature, does it MUST support Windows in this release? Do we run unit tests on Windows?
    
    There is an recipe for it: http://code.activestate.com/recipes/578513-get-memory-usage-of-windows-processes-using-getpro/, but I'm not sure it cover which versions of Windows.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15268757
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        d, c, batch = self.data, 0, self.batch
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeValues(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.partitions
    --- End diff --
    
    Can't this be negative if hash(key) is negative? That seems bad, you'd get too many elements in the map.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15299960
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1209,18 +1227,44 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
     
             # Transferring O(n) objects to Java is too expensive.  Instead, we'll
             # form the hash buckets in Python, transferring O(numPartitions) objects
    -        # to Java.  Each object is a (splitNumber, [objects]) pair.
    +        # to Java. Each object is a (splitNumber, [objects]) pair.
    +        # In order to void too huge objects, the objects are grouped into chunks.
             outputSerializer = self.ctx._unbatched_serializer
     
    +        limit = (_parse_memory(self.ctx._conf.get("spark.python.worker.memory")
    +                               or "512m") / 2)
    --- End diff --
    
    I did not find out this, sorry.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15072735
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    --- End diff --
    
    This should actually rotate among storage directories in spark.local.dir. Check out how the DiskStore works in Java.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49980302
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17110/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15271189
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        d, c, batch = self.data, 0, self.batch
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeValues(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.partitions
    --- End diff --
    
    hash(key) can be negative, X % Y is always positive in Python.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15263569
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    --- End diff --
    
    Since this is a new internal file, also add it to the "exclude" section of python/epydoc.conf so it does not show up in the docs


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264608
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    --- End diff --
    
    Explain the life cycle of this; that is, when is it not empty, when is data not empty, does it get re-filled over time, etc?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49709494
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16957/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49816795
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16993/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15211142
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1247,15 +1262,16 @@ def combineLocally(iterator):
                 return combiners.iteritems()
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    +        serializer = self.ctx.serializer
    +        spill = ((self.ctx._conf.get("spark.shuffle.spill") or 'True').lower()
    +                in ('true', '1', 'yes'))
    --- End diff --
    
    In Python, it's spelled as "True", user will try to call conf.set("spark.shuffle.spill", True), then he/she will get u"True", so the lower() will be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15078890
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    +        self.item_limit = None
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def used_memory(self):
    +        rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +        if platform.system() == 'Linux':
    +            rss >>= 10
    +        elif platform.system() == 'Darwin':
    +            rss >>= 20
    +        return rss
    +
    +    def merge(self, iterator):
    +        iterator = iter(iterator)
    +        d = self.data
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +
    +            if self.item_limit is not None:
    +                continue
    +
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory() > self.memory_limit:
    +                self.item_limit = c
    +                self._first_spill()
    +                self._partitioned_merge(iterator)
    +                return
    +
    +    def _partitioned_merge(self, iterator):
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            d = self.pdata[hash(k) % self.PARTITIONS]
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +            c += 1
    +            if c >= self.item_limit:
    +                self._spill()
    +                c = 0
    +
    +    def _first_spill(self):
    +        path = os.path.join(self.path, str(self.spills))
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    --- End diff --
    
    The only constraint here is that it should be a library available in Python by default, so that we don't ask users to install external packages. Maybe if it's not available, we can fall back to GZIP with compression level 1, or to no compression.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207775
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    --- End diff --
    
    Probably better to just call `get_used_memory()` all the time instead of making this a property, to indicate that it's expensive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207731
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1247,15 +1262,16 @@ def combineLocally(iterator):
                 return combiners.iteritems()
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    +        serializer = self.ctx.serializer
    +        spill = ((self.ctx._conf.get("spark.shuffle.spill") or 'True').lower()
    +                in ('true', '1', 'yes'))
    +        memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory") or "512m")
             def _mergeCombiners(iterator):
    -            combiners = {}
    -            for (k, v) in iterator:
    -                if k not in combiners:
    -                    combiners[k] = v
    -                else:
    -                    combiners[k] = mergeCombiners(combiners[k], v)
    -            return combiners.iteritems()
    +            merger = ExternalHashMapMerger(mergeCombiners, memory, serializer)\
    +                         if spill else MapMerger(mergeCombiners)
    --- End diff --
    
    Add a space before the `\` to match our code style in other places


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49967247
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17094/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266539
  
    --- Diff: python/pyspark/tests.py ---
    @@ -47,6 +48,64 @@
     SPARK_HOME = os.environ["SPARK_HOME"]
     
     
    +class TestMerger(unittest.TestCase):
    +
    +    def setUp(self):
    +        self.N = 1 << 16
    +        self.l = [i for i in xrange(self.N)]
    +        self.data = zip(self.l, self.l)
    +        self.agg = Aggregator(lambda x: [x], 
    +                lambda x, y: x.append(y) or x,
    +                lambda x, y: x.extend(y) or x)
    +        ExternalMerger.PARTITIONS = 8
    +        ExternalMerger.BATCH = 1 << 14
    --- End diff --
    
    Good point!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264180
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    --- End diff --
    
    Just to be clear, this recursing happens at most one time, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15263661
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    --- End diff --
    
    Call the fields of this createCombiner, mergeValue and mergeCombiners, same as in Scala. Also explain what it means for mergeCombiners to be None.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49928744
  
    Ah NM. Jenkins, test this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50101034
  
    QA results for PR 1460:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17152/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49647944
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16918/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15267225
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    --- End diff --
    
    is mergeValuers better? because this class is called Merger,  so merger.mergeValues() will more readable.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49286439
  
    QA results for PR 1460:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class Merger(object):<br>class AutoSerializer(FramedSerializer):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16772/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/1460


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49917537
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17053/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49969141
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17094/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49670702
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Merger(object):<br>class MapMerger(Merger):<br>class ExternalHashMapMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16919/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15268717
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    Can you explain why this is here? I still didn't understand why you don't always use self.memory_limit (or say self.memory_limit * 0.9 or something). Who cares if the memory is not 0 after you release it, you should still ramp up all the way before spilling.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266964
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    --- End diff --
    
    scale is unique for each ExternalMerger, the scale of it's child ExternalMerger will be multiplied by PARTITIONS.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15299925
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    --- End diff --
    
    SPARK_LOCAL_DIR will be consistent with spark.local.dir


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15325695
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,433 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    --- End diff --
    
    Unfortunately memory_info only works in psutil 2.0. I tried the Anaconda Python distribution on Mac, which has psutil 1.2.1, and it doesn't work there. In there you have to use get_memory_info() instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15306941
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    There is huge performance difference, even in doctests, so I rollbacked again.
    
    time python pyspark/shuffle.py
    
    real	0m0.583s
    user	0m0.488s
    sys	0m0.093s
    
    $ time python pyspark/shuffle.py
    
    real	0m20.674s
    user	0m6.119s
    sys	0m13.993s


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15302973
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1207,20 +1225,49 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
             if numPartitions is None:
                 numPartitions = self._defaultReducePartitions()
     
    -        # Transferring O(n) objects to Java is too expensive.  Instead, we'll
    -        # form the hash buckets in Python, transferring O(numPartitions) objects
    -        # to Java.  Each object is a (splitNumber, [objects]) pair.
    +        # Transferring O(n) objects to Java is too expensive.
    +        # Instead, we'll form the hash buckets in Python,
    +        # transferring O(numPartitions) objects to Java.
    +        # Each object is a (splitNumber, [objects]) pair.
    +        # In order to void too huge objects, the objects are
    --- End diff --
    
    Small typo: void -> avoid


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15268733
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        c, batch = 0, self.BATCH
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_combine(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.PARTITIONS
    --- End diff --
    
    (Unless you wanted to have negative hash codes too, but that's kind of weird)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49927437
  
    The last commit has fixed the tests, should run it again.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50102382
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17153/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274753
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    I see, could we force GC instead with gc.collect()? That would be cleaner if it worked. This approach seems kind of troublesome, e.g. we might ramp up to much more than the limit if we're not careful.
    
    Another option is to stop checking for more than 1000 items. You can estimate the number of items after which you should begin checking again by looking a how many items you processed until the first time you had to spill.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264650
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    --- End diff --
    
    Just so you know, our line length limit is 100 characters, not 80. You can make some of these things one line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15072860
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    --- End diff --
    
    Probably need to make the batch size an argument and pass in SparkContext._batchSize


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15300037
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        d, c, batch = self.data, 0, self.batch
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeValues(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.partitions
    +
    +    def _partitioned_mergeValues(self, iterator, limit=0):
    +        """ partition the items by key, then combine them """
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch
    +
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self.next_limit()
    +
    +    def mergeCombiners(self, iterator, check=True):
    +        """ merge (K,V) pair by mergeCombiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.agg.mergeCombiners, self.batch
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeCombiners(iterator, self.next_limit())
    +                break
    +
    +    def _partitioned_mergeCombiners(self, iterator, limit=0):
    +        """ partition the items by key, then merge them """
    +        comb, pdata = self.agg.mergeCombiners, self.pdata
    +        c, hfun = 0, self._partition
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % self.batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self.next_limit()
    +
    +    def _first_spill(self):
    +        """
    +        dump all the data into disks partition by partition.
    +
    +        The data has not been partitioned, it will iterator the
    +        dataset once, write them into different files, has no
    +        additional memory. It only called when the memory goes
    +        above limit at the first time.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        # open all the files for writing
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.partitions)]
    +
    +        for k, v in self.data.iteritems():
    +            h = self._partition(k)
    +            # put one item in batch, make it compatitable with load_stream
    +            # it will increase the memory if dump them in batch
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.partitions)]
    +        self.spills += 1
    +
    +    def _spill(self):
    +        """
    +        dump already partitioned data into disks.
    +
    +        It will dump the data in batch for better performance.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +
    +        for i in range(self.partitions):
    +            p = os.path.join(path, str(i))
    +            with open(p, "w") as f:
    +                # dump items in batch
    +                self.serializer.dump_stream(self.pdata[i].iteritems(), f)
    +            self.pdata[i].clear()
    +        self.spills += 1
    +
    +    def iteritems(self):
    +        """ return all merged items as iterator """
    +        if not self.pdata and not self.spills:
    +            return self.data.iteritems()
    +        return self._external_items()
    +
    +    def _external_items(self):
    +        """ return all partitioned items as iterator """
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        hard_limit = self.next_limit()
    +
    +        try:
    +            for i in range(self.partitions):
    +                self.data = {}
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    p = os.path.join(path, str(i))
    +                    # do not check memory during merging
    +                    self.mergeCombiners(self.serializer.load_stream(open(p)),
    +                                        False)
    +
    +                    if get_used_memory() > hard_limit and j < self.spills - 1:
    +                        self.data.clear() # will read from disk again
    +                        for v in self._recursive_merged_items(i):
    +                            yield v
    +                        return
    +
    +                for v in self.data.iteritems():
    +                    yield v
    +                self.data.clear()
    +        finally:
    +            self._cleanup()
    +
    +    def _cleanup(self):
    +        """ clean up all the files in disks """
    +        for d in self.localdirs:
    +            shutil.rmtree(d, True)
    --- End diff --
    
    It will remove the older files for each recursive spill. Also, I will remove the files after merge each partition.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15263760
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    --- End diff --
    
    Call this combineValues


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15136736
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    +        self.item_limit = None
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def used_memory(self):
    +        rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +        if platform.system() == 'Linux':
    +            rss >>= 10
    +        elif platform.system() == 'Darwin':
    +            rss >>= 20
    --- End diff --
    
    In most cases, the merger will work in no-spill mode. So it's better to fallback into no-spill mode when in Windows, an show an warning.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15300603
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    --- End diff --
    
    Good point, I add limit as the total partittions (4096 as default).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49491608
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Merger(object):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16835/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49976553
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17110/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15303331
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1265,26 +1312,28 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
             if numPartitions is None:
                 numPartitions = self._defaultReducePartitions()
     
    +        serializer = self.ctx.serializer
    +        spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower()
    +                 == 'true')
    +        memory = (_parse_memory(self.ctx._conf.get(
    +                    "spark.python.worker.memory","512m")
    --- End diff --
    
    Space after ","


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15072948
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1247,15 +1366,12 @@ def combineLocally(iterator):
                 return combiners.iteritems()
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    +        executorMemory = self.ctx._jsc.sc().executorMemory()
             def _mergeCombiners(iterator):
    -            combiners = {}
    -            for (k, v) in iterator:
    -                if k not in combiners:
    -                    combiners[k] = v
    -                else:
    -                    combiners[k] = mergeCombiners(combiners[k], v)
    -            return combiners.iteritems()
    +            merger = Merger(mergeCombiners, executorMemory * 0.7)
    --- End diff --
    
    We probably want to use the same memory as the Java append only map, which is spark.shuffle.memoryFraction * executorMemory. Check how that gets created in ExternalAppendOnlyMap


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274766
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    BTW I'd prefer to enforce the limit right now, the problem is that in environments like YARN your job just gets killed if it goes past it. It's better to have a low setting and ask users to increase it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15263824
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    --- End diff --
    
    Call this combineCombiners


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15078093
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    +        self.item_limit = None
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def used_memory(self):
    +        rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +        if platform.system() == 'Linux':
    +            rss >>= 10
    +        elif platform.system() == 'Darwin':
    +            rss >>= 20
    +        return rss
    +
    +    def merge(self, iterator):
    +        iterator = iter(iterator)
    +        d = self.data
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +
    +            if self.item_limit is not None:
    +                continue
    +
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory() > self.memory_limit:
    +                self.item_limit = c
    +                self._first_spill()
    +                self._partitioned_merge(iterator)
    +                return
    +
    +    def _partitioned_merge(self, iterator):
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            d = self.pdata[hash(k) % self.PARTITIONS]
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +            c += 1
    +            if c >= self.item_limit:
    +                self._spill()
    +                c = 0
    +
    +    def _first_spill(self):
    +        path = os.path.join(self.path, str(self.spills))
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    --- End diff --
    
    Because the data was compressed and decompressed once, so it's better to use lightweight compress method, such as LZ4/Snappy/LZO. Personally, I prefer LZ4, because it has similar compress ratio but with much higher performance than Snappy and LZO, LZF.
    
    I will add them as part of BatchedSerializer.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49271938
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16772/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15304302
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1207,20 +1225,49 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
             if numPartitions is None:
                 numPartitions = self._defaultReducePartitions()
     
    -        # Transferring O(n) objects to Java is too expensive.  Instead, we'll
    -        # form the hash buckets in Python, transferring O(numPartitions) objects
    -        # to Java.  Each object is a (splitNumber, [objects]) pair.
    +        # Transferring O(n) objects to Java is too expensive.
    +        # Instead, we'll form the hash buckets in Python,
    +        # transferring O(numPartitions) objects to Java.
    +        # Each object is a (splitNumber, [objects]) pair.
    +        # In order to void too huge objects, the objects are
    +        # grouped into chunks.
             outputSerializer = self.ctx._unbatched_serializer
     
    +        limit = (_parse_memory(self.ctx._conf.get(
    +                    "spark.python.worker.memory", "512m") / 2)
    --- End diff --
    
    The overhead of splitting the items into small chunks is MUCH lower than that in ExternalMerger, so this limit is smaller than ExternalMerger (0.5 < 0.9). In the worst cases, it will yield all the items after every `batch` number of items.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15333099
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,436 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        process = psutil.Process(os.getpid())
    +        if hasattr(process, "memory_info"):
    +            info = process.memory_info()
    +        else:
    +            info = process.get_memory_info()
    +        return info.rss >> 20
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("Please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    Merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ Combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ Merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ Return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ Combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ Merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ Return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    # the max total partitions created recursively
    +    MAX_TOTAL_PARTITIONS = 4096
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data, list of dicts
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ Get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ Choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def _next_limit(self):
    +        """
    +        Return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def mergeValues(self, iterator):
    +        """ Combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        d, c, batch = self.data, 0, self.batch
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeValues(iterator, self._next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ Return the partition for key """
    +        return (hash(key) / self.scale) % self.partitions
    +
    +    def _partitioned_mergeValues(self, iterator, limit=0):
    +        """ Partition the items by key, then combine them """
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch
    +
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self._next_limit()
    +
    +    def mergeCombiners(self, iterator, check=True):
    +        """ Merge (K,V) pair by mergeCombiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.agg.mergeCombiners, self.batch
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeCombiners(iterator, self._next_limit())
    +                break
    +
    +    def _partitioned_mergeCombiners(self, iterator, limit=0):
    +        """ Partition the items by key, then merge them """
    +        comb, pdata = self.agg.mergeCombiners, self.pdata
    +        c, hfun = 0, self._partition
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % self.batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self._next_limit()
    +
    +    def _first_spill(self):
    +        """
    +        Dump all the data into disks partition by partition.
    +
    +        The data has not been partitioned, it will iterator the
    +        dataset once, write them into different files, has no
    +        additional memory. It only called when the memory goes
    +        above limit at the first time.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        # open all the files for writing
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.partitions)]
    +
    +        for k, v in self.data.iteritems():
    +            h = self._partition(k)
    +            # put one item in batch, make it compatitable with load_stream
    +            # it will increase the memory if dump them in batch
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.partitions)]
    +        self.spills += 1
    +
    +    def _spill(self):
    +        """
    +        dump already partitioned data into disks.
    +
    +        It will dump the data in batch for better performance.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +
    +        for i in range(self.partitions):
    +            p = os.path.join(path, str(i))
    +            with open(p, "w") as f:
    +                # dump items in batch
    +                self.serializer.dump_stream(self.pdata[i].iteritems(), f)
    +            self.pdata[i].clear()
    +        self.spills += 1
    +
    +    def iteritems(self):
    +        """ Return all merged items as iterator """
    +        if not self.pdata and not self.spills:
    +            return self.data.iteritems()
    +        return self._external_items()
    +
    +    def _external_items(self):
    +        """ Return all partitioned items as iterator """
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        hard_limit = self._next_limit()
    +
    +        try:
    +            for i in range(self.partitions):
    +                self.data = {}
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    p = os.path.join(path, str(i))
    +                    # do not check memory during merging
    +                    self.mergeCombiners(self.serializer.load_stream(open(p)),
    +                                        False)
    +
    +                    # limit the total partitions
    +                    if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS
    +                            and j < self.spills - 1
    +                            and get_used_memory() > hard_limit):
    +                        self.data.clear() # will read from disk again
    +                        for v in self._recursive_merged_items(i):
    +                            yield v
    +                        return
    +
    +                for v in self.data.iteritems():
    +                    yield v
    +                self.data.clear()
    +
    +                # remove the merged partition
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    os.remove(os.path.join(path, str(i)))
    +
    +        finally:
    +            self._cleanup()
    +
    +    def _cleanup(self):
    +        """ Clean up all the files in disks """
    +        for d in self.localdirs:
    +            shutil.rmtree(d, True)
    +
    +    def _recursive_merged_items(self, start):
    +        """
    +        merge the partitioned items and return the as iterator
    +
    +        If one partition can not be fit in memory, then them will be
    +        partitioned and merged recursively.
    +        """
    +        # make sure all the data are dumps into disks.
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        assert self.spills > 0
    +
    +        for i in range(start, self.partitions):
    +            subdirs = [os.path.join(d, "parts", str(i))
    +                            for d in self.localdirs]
    +            m = ExternalMerger(self.agg, self.memory_limit, self.serializer,
    +                    subdirs, self.scale * self.partitions)
    +            m.pdata = [{} for _ in range(self.partitions)]
    +            limit = self._next_limit()
    +
    +            for j in range(self.spills):
    +                path = self._get_spill_dir(j)
    +                p = os.path.join(path, str(i))
    +                m._partitioned_mergeCombiners(
    +                        self.serializer.load_stream(open(p)))
    +
    +                if get_used_memory() > limit:
    +                    m._spill()
    +                    limit = self._next_limit()
    +
    +            for v in m._external_items():
    +                yield v
    +
    +            # remove the merged partition
    +            for j in range(self.spills):
    +                path = self._get_spill_dir(j)
    +                os.remove(os.path.join(path, str(i)))
    +
    +
    +if __name__ == "__main__":
    +    import doctest
    +    doctest.testmod()
    --- End diff --
    
    If you want these unit tests to be run by Jenkins, you need to also call this file in `python/run-tests`. Seems worthwhile since there are some tests in ExternalMerger.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266298
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    --- End diff --
    
    We already have an warning to tell people that they should install psutil then they got spilling.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49805468
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16991/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15208070
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    --- End diff --
    
    Add a high-level description here about how the class works and what the main variables and methods are. For example take a look at the class comment in https://github.com/apache/spark/pull/1499/files#diff-7.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15188798
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +169,18 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +def _parse_memory(s):
    --- End diff --
    
    Add a comment to this saying it returns a number in MB


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49814922
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16991/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266361
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    --- End diff --
    
    This recursing could happen many times, if the data sets are really huge.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49680209
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Merger(object):<br>class MapMerger(Merger):<br>class ExternalHashMapMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16929/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15303224
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1207,20 +1225,49 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
             if numPartitions is None:
                 numPartitions = self._defaultReducePartitions()
     
    -        # Transferring O(n) objects to Java is too expensive.  Instead, we'll
    -        # form the hash buckets in Python, transferring O(numPartitions) objects
    -        # to Java.  Each object is a (splitNumber, [objects]) pair.
    +        # Transferring O(n) objects to Java is too expensive.
    +        # Instead, we'll form the hash buckets in Python,
    +        # transferring O(numPartitions) objects to Java.
    +        # Each object is a (splitNumber, [objects]) pair.
    +        # In order to void too huge objects, the objects are
    +        # grouped into chunks.
             outputSerializer = self.ctx._unbatched_serializer
     
    +        limit = (_parse_memory(self.ctx._conf.get(
    +                    "spark.python.worker.memory", "512m") / 2)
    --- End diff --
    
    Could you get the same problem of increasing the limit here that you have in ExternalMerger? (If Python doesn't free memory right away)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15072768
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    +        self.item_limit = None
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def used_memory(self):
    +        rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +        if platform.system() == 'Linux':
    +            rss >>= 10
    +        elif platform.system() == 'Darwin':
    +            rss >>= 20
    --- End diff --
    
    We also need to make it work on Windows, or at least fail gracefully. Do you know how to get this number on Windows? You can spin up a Windows machine on EC2 to try it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207630
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    --- End diff --
    
    Can we actually use this by default yet or will it fail for NumPy arrays? If it won't work by default, we should use PickleSerializer instead and wait to fix this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15208014
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    --- End diff --
    
    Better to call this something like "python" instead.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49788328
  
    QA results for PR 1460:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16975/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15073011
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    +    """
    +    Choose marshal or cPickle as serialization protocol autumatically
    +    """
    +    def __init__(self):
    +        FramedSerializer.__init__(self)
    +        self._type = None
    +
    +    def dumps(self, obj):
    +        try:
    +            if self._type is not None:
    +                raise TypeError("fallback")
    +            return 'M' + marshal.dumps(obj)
    +        except Exception:
    +            self._type = 'P'
    +            return 'P' + cPickle.dumps(obj, -1)
    --- End diff --
    
    If the objects are not marshal-able but are pickle-able, is there a big performance cost to throwing an exception on each write? Would be good to test this, because if not, we can make this serializer our default where we now use Pickle. Even if there is a cost maybe we can do something where if 10% of the objects written fail to marshal we switch to always using pickle.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207622
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +169,20 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +def _parse_memory(s):
    +    """
    +    It returns a number in MB
    --- End diff --
    
    Say something slightly longer, e.g. "Parse a memory string in the format supported by Java (e.g. 1g, 200m) and return the value in MB"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264473
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    --- End diff --
    
    Instead of making these class variables, make them optional parameters to the constructor


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50110575
  
    Thanks Davies. I've merged this in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49904887
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17043/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207721
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1247,15 +1262,16 @@ def combineLocally(iterator):
                 return combiners.iteritems()
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    +        serializer = self.ctx.serializer
    +        spill = ((self.ctx._conf.get("spark.shuffle.spill") or 'True').lower()
    +                in ('true', '1', 'yes'))
    +        memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory") or "512m")
             def _mergeCombiners(iterator):
    -            combiners = {}
    -            for (k, v) in iterator:
    -                if k not in combiners:
    -                    combiners[k] = v
    -                else:
    -                    combiners[k] = mergeCombiners(combiners[k], v)
    -            return combiners.iteritems()
    +            merger = ExternalHashMapMerger(mergeCombiners, memory, serializer)\
    +                         if spill else MapMerger(mergeCombiners)
    +            merger.merge(iterator)
    +            return merger.iteritems()
             return shuffled.mapPartitions(_mergeCombiners)
    --- End diff --
    
    This only implements external merging in the reduce tasks, but we need it in the map tasks too. For that you'll need to modify the Merger interface to take `createCombiner`, `mergeValue` and `mergeCombiners` together. Please add those there and add tests for the richer interface.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49928790
  
    Ah never mind.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207837
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    --- End diff --
    
    Also I don't really understand the purpose of this, why don't we always go up to the same limit?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207593
  
    --- Diff: docs/configuration.md ---
    @@ -195,6 +195,15 @@ Apart from these, the following properties are also available, and may be useful
         Spark's dependencies and user dependencies. It is currently an experimental feature.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.python.worker.memory</code></td>
    +  <td>512m</td>
    +  <td>
    +    Amount of memory to use per python worker process during aggregation, in the same
    +    format as JVM memory strings (e.g. <code>512m</code>, <code>2g</code>). If the memory
    +    used during aggregation go above this amount, it will spill the data into disks.
    --- End diff --
    
    Small typo: go -> goes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15137187
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    +    """
    +    Choose marshal or cPickle as serialization protocol autumatically
    +    """
    +    def __init__(self):
    +        FramedSerializer.__init__(self)
    +        self._type = None
    +
    +    def dumps(self, obj):
    +        try:
    +            if self._type is not None:
    +                raise TypeError("fallback")
    +            return 'M' + marshal.dumps(obj)
    +        except Exception:
    +            self._type = 'P'
    +            return 'P' + cPickle.dumps(obj, -1)
    +
    +    def loads(self, stream):
    +        _type = stream[0]
    +        if _type == 'M':
    +            return marshal.loads(stream[1:])
    +        elif _type == 'P':
    +            return cPickle.loads(stream[1:])
    --- End diff --
    
    the input of loads() is an string, I had update the name of argument.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mattf <gi...@git.apache.org>.
Github user mattf commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15286571
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1265,26 +1309,26 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners,
             if numPartitions is None:
                 numPartitions = self._defaultReducePartitions()
     
    +        serializer = self.ctx.serializer
    +        spill = (self.ctx._conf.get("spark.shuffle.spill") or 'True').lower() == 'true'
    +        memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory") or "512m")
    +        agg = Aggregator(createCombiner, mergeValue, mergeCombiners)
    +
             def combineLocally(iterator):
    -            combiners = {}
    -            for x in iterator:
    -                (k, v) = x
    -                if k not in combiners:
    -                    combiners[k] = createCombiner(v)
    -                else:
    -                    combiners[k] = mergeValue(combiners[k], v)
    -            return combiners.iteritems()
    +            merger = ExternalMerger(agg, memory, serializer) \
    +                         if spill else InMemoryMerger(agg)
    +            merger.mergeValues(iterator)
    +            return merger.iteritems()
    +
             locally_combined = self.mapPartitions(combineLocally)
             shuffled = locally_combined.partitionBy(numPartitions)
    -
    + 
    --- End diff --
    
    stray whitespace


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15137226
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    +    """
    +    Choose marshal or cPickle as serialization protocol autumatically
    +    """
    +    def __init__(self):
    +        FramedSerializer.__init__(self)
    +        self._type = None
    +
    +    def dumps(self, obj):
    +        try:
    +            if self._type is not None:
    +                raise TypeError("fallback")
    +            return 'M' + marshal.dumps(obj)
    +        except Exception:
    +            self._type = 'P'
    +            return 'P' + cPickle.dumps(obj, -1)
    --- End diff --
    
    I had add an fast path for it, no exception cost any more.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264236
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    --- End diff --
    
    Just to be clear, this recursing happens at most one time, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207983
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    +
    +    def merge(self, iterator, check=True):
    +        """ merge (K,V) pair by combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.combiner, self.BATCH
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and self.used_memory > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_merge(iterator, self.next_limit)
    +                break
    +
    +    def _hash(self, key):
    +        return (hash(key) / self.scale) % self.PARTITIONS
    +
    +    def _partitioned_merge(self, iterator, limit):
    +        comb, pdata, hfun = self.combiner, self.pdata, self._hash
    +        c = 0
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory > limit:
    +                self._spill()
    +                limit = self.next_limit
    +
    +    def _first_spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    +        for k, v in self.data.iteritems():
    +            h = self._hash(k)
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.PARTITIONS)]
    +        self.spills += 1
    +
    +    def _spill(self):
    --- End diff --
    
    It's weird that this is so similar to `_spill`. Is there a way to make them the same? If not, can you give them more descriptive names and a comment that explains when each one is called (i.e. that the second one is when we recurse)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15271374
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    --- End diff --
    
    These are explained in the docs of ExternalMerger.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49540306
  
    It looks like pushing a new rebased commit hid my comments, but click on them above to make sure you see them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15303079
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,432 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("Please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    Merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ Combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ Merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ Return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ Combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ Merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ Return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    TOTAL_PARTITIONS = 4096
    --- End diff --
    
    Call it MAX_TOTAL_PARTITIONS to make it clearer and add a comment above it


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274877
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        d, c, batch = self.data, 0, self.batch
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeValues(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.partitions
    +
    +    def _partitioned_mergeValues(self, iterator, limit=0):
    +        """ partition the items by key, then combine them """
    +        # speedup attribute lookup
    +        creator, comb = self.agg.createCombiner, self.agg.mergeValue
    +        c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch
    +
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self.next_limit()
    +
    +    def mergeCombiners(self, iterator, check=True):
    +        """ merge (K,V) pair by mergeCombiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.agg.mergeCombiners, self.batch
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_mergeCombiners(iterator, self.next_limit())
    +                break
    +
    +    def _partitioned_mergeCombiners(self, iterator, limit=0):
    +        """ partition the items by key, then merge them """
    +        comb, pdata = self.agg.mergeCombiners, self.pdata
    +        c, hfun = 0, self._partition
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +
    +            c += 1
    +            if c % self.batch == 0 and get_used_memory() > limit:
    +                self._spill()
    +                limit = self.next_limit()
    +
    +    def _first_spill(self):
    +        """
    +        dump all the data into disks partition by partition.
    +
    +        The data has not been partitioned, it will iterator the
    +        dataset once, write them into different files, has no
    +        additional memory. It only called when the memory goes
    +        above limit at the first time.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        # open all the files for writing
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.partitions)]
    +
    +        for k, v in self.data.iteritems():
    +            h = self._partition(k)
    +            # put one item in batch, make it compatitable with load_stream
    +            # it will increase the memory if dump them in batch
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.partitions)]
    +        self.spills += 1
    +
    +    def _spill(self):
    +        """
    +        dump already partitioned data into disks.
    +
    +        It will dump the data in batch for better performance.
    +        """
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +
    +        for i in range(self.partitions):
    +            p = os.path.join(path, str(i))
    +            with open(p, "w") as f:
    +                # dump items in batch
    +                self.serializer.dump_stream(self.pdata[i].iteritems(), f)
    +            self.pdata[i].clear()
    +        self.spills += 1
    +
    +    def iteritems(self):
    +        """ return all merged items as iterator """
    +        if not self.pdata and not self.spills:
    +            return self.data.iteritems()
    +        return self._external_items()
    +
    +    def _external_items(self):
    +        """ return all partitioned items as iterator """
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        hard_limit = self.next_limit()
    +
    +        try:
    +            for i in range(self.partitions):
    +                self.data = {}
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    p = os.path.join(path, str(i))
    +                    # do not check memory during merging
    +                    self.mergeCombiners(self.serializer.load_stream(open(p)),
    +                                        False)
    +
    +                    if get_used_memory() > hard_limit and j < self.spills - 1:
    +                        self.data.clear() # will read from disk again
    +                        for v in self._recursive_merged_items(i):
    +                            yield v
    +                        return
    +
    +                for v in self.data.iteritems():
    +                    yield v
    +                self.data.clear()
    +        finally:
    +            self._cleanup()
    +
    +    def _cleanup(self):
    +        """ clean up all the files in disks """
    +        for d in self.localdirs:
    +            shutil.rmtree(d, True)
    --- End diff --
    
    When we spill multiple times, would it be possible to remove the older files for each recursive spill before we're done iterating through everything?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207812
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    --- End diff --
    
    Same here, make this a method, it's confusing for it to be a property


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15211059
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -57,7 +57,9 @@ private[spark] class PythonRDD[T: ClassTag](
       override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
         val startTime = System.currentTimeMillis
         val env = SparkEnv.get
    -    val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap)
    +    val localdir = env.conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))
    +    val worker: Socket = env.createPythonWorker(pythonExec,
    +      envVars.toMap + ("SPARK_LOCAL_DIR" -> localdir))
    --- End diff --
    
    DiskBlockManager.localDirs is private, make it public?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15211313
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    --- End diff --
    
    It will fails in some cases, such as array, so it's not safe to make it as default. I will improve it later and try to make it as default. Currently, it's still useful, because people can use it in most cases.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49484674
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16835/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50099568
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17153/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49346100
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class Merger(object):<br>class AutoSerializer(FramedSerializer):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16782/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mattf <gi...@git.apache.org>.
Github user mattf commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15286548
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -1209,18 +1227,44 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash):
     
             # Transferring O(n) objects to Java is too expensive.  Instead, we'll
             # form the hash buckets in Python, transferring O(numPartitions) objects
    -        # to Java.  Each object is a (splitNumber, [objects]) pair.
    +        # to Java. Each object is a (splitNumber, [objects]) pair.
    +        # In order to void too huge objects, the objects are grouped into chunks.
             outputSerializer = self.ctx._unbatched_serializer
     
    +        limit = (_parse_memory(self.ctx._conf.get("spark.python.worker.memory")
    +                               or "512m") / 2)
    --- End diff --
    
    instead of 'conf.get("xyz") or "default"' why not use 'conf.get("xyz", "default")'?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15208037
  
    --- Diff: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala ---
    @@ -57,7 +57,9 @@ private[spark] class PythonRDD[T: ClassTag](
       override def compute(split: Partition, context: TaskContext): Iterator[Array[Byte]] = {
         val startTime = System.currentTimeMillis
         val env = SparkEnv.get
    -    val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap)
    +    val localdir = env.conf.get("spark.local.dir", System.getProperty("java.io.tmpdir"))
    +    val worker: Socket = env.createPythonWorker(pythonExec,
    +      envVars.toMap + ("SPARK_LOCAL_DIR" -> localdir))
    --- End diff --
    
    Instead of passing spark.local.dir, we should figure out which directories the DiskBlockManager created (you can get it from `env.blockManager`) and pass a comma-separated list of those. This way the data for this Spark application is all in one directory, and Java can make sure we clean it all up at the end. Otherwise the way you have things set up now, those directories are never cleared if the Python worker crashes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50098449
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17152/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207683
  
    --- Diff: python/pyspark/tests.py ---
    @@ -47,6 +48,40 @@
     SPARK_HOME = os.environ["SPARK_HOME"]
     
     
    +class TestMerger(unittest.TestCase):
    +
    +    def setUp(self):
    +        self.N = 1<<18
    +        self.l = [i for i in xrange(self.N)]
    +        self.data = zip(self.l, self.l)
    +        ExternalHashMapMerger.PARTITIONS = 8
    +        ExternalHashMapMerger.BATCH = 1<<14
    +
    +    def test_in_memory(self):
    +        m = MapMerger(lambda x,y: x+y)
    +        m.merge(self.data)
    +        self.assertEqual(sum(v for k,v in m.iteritems()), sum(xrange(self.N)))
    --- End diff --
    
    Put spaces around the operators (e.g. `,`, `+`, etc)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15271462
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    It just try to avoid the performance regression in normal cases. The memory preserved for Python worker is undefined, but it's limited to 512M in this patch. Some job may have performance regression if we have too STRICT memory limitation in this patch.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15073144
  
    --- Diff: python/pyspark/serializers.py ---
    @@ -297,6 +297,33 @@ class MarshalSerializer(FramedSerializer):
         loads = marshal.loads
     
     
    +class AutoSerializer(FramedSerializer):
    +    """
    +    Choose marshal or cPickle as serialization protocol autumatically
    +    """
    +    def __init__(self):
    +        FramedSerializer.__init__(self)
    +        self._type = None
    +
    +    def dumps(self, obj):
    +        try:
    +            if self._type is not None:
    +                raise TypeError("fallback")
    +            return 'M' + marshal.dumps(obj)
    +        except Exception:
    +            self._type = 'P'
    +            return 'P' + cPickle.dumps(obj, -1)
    +
    +    def loads(self, stream):
    +        _type = stream[0]
    +        if _type == 'M':
    +            return marshal.loads(stream[1:])
    +        elif _type == 'P':
    +            return cPickle.loads(stream[1:])
    --- End diff --
    
    Seems to be more efficient to read a byte with stream.read(1) than to index into the stream, but I'm not sure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49917693
  
    QA results for PR 1460:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class SimpleAggregator(Aggregator):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17043/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274557
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    --- End diff --
    
    It's better to recap here. Also, say their format, specifically that pdata will contain a separate map for each partition ID, since we already know the partition ID.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207936
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    +
    +    def merge(self, iterator, check=True):
    --- End diff --
    
    Explain the arguments to this (in particular check should be called something else, e.g. `canRecurse`, since it's really checking whether we can spill further)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15268727
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        c, batch = 0, self.BATCH
    +
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +            c += 1
    +            if c % batch == 0 and get_used_memory() > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_combine(iterator, self.next_limit())
    +                break
    +
    +    def _partition(self, key):
    +        """ return the partition for key """
    +        return (hash(key) / self.scale) % self.PARTITIONS
    --- End diff --
    
    Can't this be negative if the key is negative? Add a test for that and fix it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49658257
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16919/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274517
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    --- End diff --
    
    I see, in that case you should add a limit, because I can see it getting into an infinite loop. For example imagine that one hash code has too many items -- you'd keep loading and unloading stuff from disk.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15271358
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    When objects are freed in Python, the memory may be not released to system immediately. For example, given limit=512M, but the used memory goes up to 550M until it find out this. After spilling, the used memory is still 520M (300MB of them is free in Python). If we still use limit=512M, then after merging 1000 objects, it will dump the data into disks. If we use limit=530M, then it will dump the objects until them occupied all the free memory in Python. The later approach will have better performance.
    
    This will be especially useful when people have small spark.python.work.memory but huge memory in the machine. 
    
    How do you think of it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15325703
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,433 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ Return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    --- End diff --
    
    Also, maybe don't call the process "self", it's kind of confusing since it sounds like a "this" object


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15300412
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    +
    +    def _get_spill_dir(self, n):
    +        """ choose one directory for spill by number n """
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    def next_limit(self):
    +        """
    +        return the next memory limit. If the memory is not released
    +        after spilling, it will dump the data only when the used memory
    +        starts to increase.
    +        """
    +        return max(self.memory_limit, get_used_memory() * 1.05)
    --- End diff --
    
    Even with gc.collect(), Python may still keep the memory.
    
    OK, I will rollback this. We will see there is performance regression or not.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264487
  
    --- Diff: python/pyspark/tests.py ---
    @@ -47,6 +48,64 @@
     SPARK_HOME = os.environ["SPARK_HOME"]
     
     
    +class TestMerger(unittest.TestCase):
    +
    +    def setUp(self):
    +        self.N = 1 << 16
    +        self.l = [i for i in xrange(self.N)]
    +        self.data = zip(self.l, self.l)
    +        self.agg = Aggregator(lambda x: [x], 
    +                lambda x, y: x.append(y) or x,
    +                lambda x, y: x.extend(y) or x)
    +        ExternalMerger.PARTITIONS = 8
    +        ExternalMerger.BATCH = 1 << 14
    --- End diff --
    
    These are never cleared, so they might affect future tests. It's better to make them arguments to the constructor of ExternalMerger.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49774956
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16975/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15268748
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        return [os.path.join(d, "python", str(os.getpid()), str(id(self)))
    +                for d in dirs]
    --- End diff --
    
    Ah OK, sure


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207668
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    --- End diff --
    
    Better to call this ExternalMerger


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15211652
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    +
    +    def merge(self, iterator, check=True):
    +        """ merge (K,V) pair by combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.combiner, self.BATCH
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and self.used_memory > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_merge(iterator, self.next_limit)
    +                break
    +
    +    def _hash(self, key):
    +        return (hash(key) / self.scale) % self.PARTITIONS
    +
    +    def _partitioned_merge(self, iterator, limit):
    +        comb, pdata, hfun = self.combiner, self.pdata, self._hash
    +        c = 0
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory > limit:
    +                self._spill()
    +                limit = self.next_limit
    +
    +    def _first_spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    +        for k, v in self.data.iteritems():
    +            h = self._hash(k)
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.PARTITIONS)]
    +        self.spills += 1
    +
    +    def _spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        for i in range(self.PARTITIONS):
    +            p = os.path.join(path, str(i))
    +            with open(p, "w") as f:
    +                self.serializer.dump_stream(self.pdata[i].iteritems(), f)
    +            self.pdata[i].clear()
    +        self.spills += 1
    +
    +    def iteritems(self):
    +        """ iterator of all merged (K,V) pairs """
    +        if not self.pdata and not self.spills:
    +            return self.data.iteritems()
    +        return self._external_items()
    +
    +    def _external_items(self):
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        hard_limit = self.next_limit
    +
    +        try:
    +            for i in range(self.PARTITIONS):
    +                self.data = {}
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    p = os.path.join(path, str(i))
    +                    self.merge(self.serializer.load_stream(open(p)), False)
    +
    +                    if self.used_memory > hard_limit and j < self.spills - 1:
    --- End diff --
    
    It will make merge() even complicated, I think it's better to check here. In most cases, One level hash is enough. If merge() return a boolean, then it will break the doc tests.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49718092
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Aggregator(object):<br>class Merger(object):<br>class InMemoryMerger(Merger):<br>class ExternalMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16957/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15274897
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    --- End diff --
    
    Small style thing: capitalize the beginning of each doc comment to match the rest of our code. (This applies to a bunch of places.)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mattf <gi...@git.apache.org>.
Github user mattf commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15286886
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,416 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        """ return the used memory in MB """
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to have better "
    +                    "support with spilling")
    +            if platform.system() == "Darwin":
    +                import resource
    +                rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +                return rss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    """
    +    Aggregator has tree functions to merge values into combiner.
    +
    +    createCombiner:  (value) -> combiner
    +    mergeValue:      (combine, value) -> combiner
    +    mergeCombiners:  (combiner, combiner) -> combiner
    +    """
    +
    +    def __init__(self, createCombiner, mergeValue, mergeCombiners):
    +        self.createCombiner = createCombiner
    +        self.mergeValue = mergeValue
    +        self.mergeCombiners = mergeCombiners
    +
    +
    +class SimpleAggregator(Aggregator):
    +
    +    """
    +    SimpleAggregator is useful for the cases that combiners have
    +    same type with values
    +    """
    +
    +    def __init__(self, combiner):
    +        Aggregator.__init__(self, lambda x: x, combiner, combiner)
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def mergeValues(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator = self.data, self.agg.createCombiner
    +        comb = self.agg.mergeValue
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def mergeCombiners(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiners
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    `data` and `pdata` are used to hold the merged items in memory.
    +    At first, all the data are merged into `data`. Once the used
    +    memory goes over limit, the items in `data` are dumped indo
    +    disks, `data` will be cleared, all rest of items will be merged
    +    into `pdata` and then dumped into disks. Before returning, all
    +    the items in `pdata` will be dumped into disks.
    +
    +    Finally, if any items were spilled into disks, each partition
    +    will be merged into `data` and be yielded, then cleared.
    +
    +    >>> agg = SimpleAggregator(lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1, partitions=64, batch=10000):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # number of partitions when spill data into disks
    +        self.partitions = partitions
    +        # check the memory after # of items merged
    +        self.batch = batch
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    +        self.pdata = []
    +        # number of chunks dumped into disks
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        """ get all the directories """
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    --- End diff --
    
    default for spark_local_dir/spark.local.dir should be /tmp for consistency


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207914
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    --- End diff --
    
    It's weird to make these class variables; just make them arguments to a constructor instead, with these default values.
    
    Also, explain what they each are here (I guess BATCH is how often we check the size?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15264616
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    --- End diff --
    
    Explain when this changes


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49334156
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16782/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15073412
  
    --- Diff: python/pyspark/rdd.py ---
    @@ -168,6 +170,123 @@ def _replaceRoot(self, value):
                 self._sink(1)
     
     
    +class Merger(object):
    +    """
    +    External merger will dump the aggregated data into disks when memory usage is above
    +    the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = Merger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> merger.spills
    +    100
    +    >>> sum(1 for k,v in merger.iteritems())
    +    10000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 1000
    +
    +    def __init__(self, combiner, memory_limit=256, path="/tmp/pyspark", serializer=None):
    +        self.combiner = combiner
    +        self.path = os.path.join(path, str(os.getpid()))
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or BatchedSerializer(AutoSerializer(), 1024)
    +        self.item_limit = None
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def used_memory(self):
    +        rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss
    +        if platform.system() == 'Linux':
    +            rss >>= 10
    +        elif platform.system() == 'Darwin':
    +            rss >>= 20
    +        return rss
    +
    +    def merge(self, iterator):
    +        iterator = iter(iterator)
    +        d = self.data
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +
    +            if self.item_limit is not None:
    +                continue
    +
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory() > self.memory_limit:
    +                self.item_limit = c
    +                self._first_spill()
    +                self._partitioned_merge(iterator)
    +                return
    +
    +    def _partitioned_merge(self, iterator):
    +        comb = self.combiner
    +        c = 0
    +        for k, v in iterator:
    +            d = self.pdata[hash(k) % self.PARTITIONS]
    +            if k in d:
    +                d[k] = comb(d[k], v)
    +            else:
    +                d[k] = v
    +            c += 1
    +            if c >= self.item_limit:
    +                self._spill()
    +                c = 0
    +
    +    def _first_spill(self):
    +        path = os.path.join(self.path, str(self.spills))
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    --- End diff --
    
    In the future we might want to compress these with GzipFile, but for now just add a TODO


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15267205
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    --- End diff --
    
    Is mergeCombiners better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15208097
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    +
    +    def merge(self, iterator, check=True):
    +        """ merge (K,V) pair by combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.combiner, self.BATCH
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and self.used_memory > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_merge(iterator, self.next_limit)
    +                break
    +
    +    def _hash(self, key):
    +        return (hash(key) / self.scale) % self.PARTITIONS
    +
    +    def _partitioned_merge(self, iterator, limit):
    +        comb, pdata, hfun = self.combiner, self.pdata, self._hash
    +        c = 0
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory > limit:
    +                self._spill()
    +                limit = self.next_limit
    +
    +    def _first_spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    +        for k, v in self.data.iteritems():
    +            h = self._hash(k)
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.PARTITIONS)]
    +        self.spills += 1
    +
    +    def _spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        for i in range(self.PARTITIONS):
    +            p = os.path.join(path, str(i))
    +            with open(p, "w") as f:
    +                self.serializer.dump_stream(self.pdata[i].iteritems(), f)
    +            self.pdata[i].clear()
    +        self.spills += 1
    +
    +    def iteritems(self):
    +        """ iterator of all merged (K,V) pairs """
    +        if not self.pdata and not self.spills:
    +            return self.data.iteritems()
    +        return self._external_items()
    +
    +    def _external_items(self):
    +        assert not self.data
    +        if any(self.pdata):
    +            self._spill()
    +        hard_limit = self.next_limit
    +
    +        try:
    +            for i in range(self.PARTITIONS):
    +                self.data = {}
    +                for j in range(self.spills):
    +                    path = self._get_spill_dir(j)
    +                    p = os.path.join(path, str(i))
    +                    self.merge(self.serializer.load_stream(open(p)), False)
    +
    +                    if self.used_memory > hard_limit and j < self.spills - 1:
    --- End diff --
    
    It's probably better to make `self.merge` return a boolean as to whether it succeeded than to check `self.used_memory > hard_limit` here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49928696
  
    Looks like the latest tested code has an error in the test suite:
    ```
    Running PySpark tests. Output is in python/unit-tests.log.
    Running test: pyspark/rdd.py
      File "pyspark/rdd.py", line 1239
        def add_shuffle_key(split, iterator):
          ^
    SyntaxError: invalid syntax
    Had test failures; see logs.
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49660273
  
    QA results for PR 1460:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds the following public classes (experimental):<br>class AutoSerializer(FramedSerializer):<br>class Merger(object):<br>class MapMerger(Merger):<br>class ExternalHashMapMerger(Merger):<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16918/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15300459
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,378 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, PickleSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Aggregator(object):
    +
    +    def __init__(self, creator, combiner, mergeCombiner=None):
    +        self.creator = creator
    +        self.combiner = combiner
    +        self.mergeCombiner = mergeCombiner or combiner
    +
    +
    +class Merger(object):
    +
    +    """
    +    merge shuffled data together by aggregator
    +    """
    +
    +    def __init__(self, aggregator):
    +        self.agg = aggregator
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        raise NotImplementedError
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        raise NotImplementedError
    +
    +
    +class InMemoryMerger(Merger):
    +
    +    """
    +    In memory merger based on in-memory dict.
    +    """
    +
    +    def __init__(self, aggregator):
    +        Merger.__init__(self, aggregator)
    +        self.data = {}
    +
    +    def combine(self, iterator):
    +        """ combine the items by creator and combiner """
    +        # speed up attributes lookup
    +        d, creator, comb = self.data, self.agg.creator, self.agg.combiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else creator(v)
    +
    +    def merge(self, iterator):
    +        """ merge the combined items by mergeCombiner """
    +        # speed up attributes lookup
    +        d, comb = self.data, self.agg.mergeCombiner
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        """ return the merged items ad iterator """
    +        return self.data.iteritems()
    +
    +
    +class ExternalMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when
    +    memory usage goes above the limit, then merge them together.
    +
    +    This class works as follows:
    +
    +    - It repeatedly combine the items and save them in one dict in 
    +      memory.
    +
    +    - When the used memory goes above memory limit, it will split
    +      the combined data into partitions by hash code, dump them
    +      into disk, one file per partition.
    +
    +    - Then it goes through the rest of the iterator, combine items
    +      into different dict by hash. Until the used memory goes over
    +      memory limit, it dump all the dicts into disks, one file per
    +      dict. Repeat this again until combine all the items.
    +
    +    - Before return any items, it will load each partition and
    +      combine them seperately. Yield them before loading next
    +      partition.
    +
    +    - During loading a partition, if the memory goes over limit,
    +      it will partition the loaded data and dump them into disks
    +      and load them partition by partition again.
    +
    +    >>> agg = Aggregator(lambda x: x, lambda x, y: x + y)
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> N = 10000
    +    >>> merger.combine(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +
    +    >>> merger = ExternalMerger(agg, 10)
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64  # number of partitions when spill data into disks
    +    BATCH = 10000  # check the memory after # of items merged
    +
    +    def __init__(self, aggregator, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        Merger.__init__(self, aggregator)
    +        self.memory_limit = memory_limit
    +        # default serializer is only used for tests
    +        self.serializer = serializer or \
    +                BatchedSerializer(PickleSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        # scale is used to scale down the hash of key for recursive hash map,
    +        self.scale = scale
    +        # unpartitioned merged data
    +        self.data = {}
    +        # partitioned merged data
    --- End diff --
    
    pdata is list of dicts, I add a comment for it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-49808272
  
    QA tests have started for PR 1460. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/16993/consoleFull


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50072351
  
    BTW here's a patch that adds the GC calls I talked about above: https://gist.github.com/mateiz/297b8618ed033e7c8005


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15207662
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    --- End diff --
    
    Better to call this InMemoryMerger


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on the pull request:

    https://github.com/apache/spark/pull/1460#issuecomment-50072248
  
    Hey Davies, I tried this out a bit and saw two issues / areas for improvement:
    
    1) Since the ExternalMerger is used in both map tasks and reduce tasks, one problem that can happen is that the reduce task's data is already hashed modulo the # of reduce tasks, and so you get many empty buckets. For example, if you have 2 reduce tasks, task 0 gets all the values whose hash code is even, so it can only use half its buckets. If you have 64 reduce tasks, only one bucket is used.
    
    The best way to fix this would be to hash values with a random hash function when choosing the bucket. One simple way might be to generate a random integer X for each ExternalMerger and then take hash((key, X)) instead of hash(key) when choosing the bucket. This is equivalent to salting your hash function. Maybe you have other ideas but I'd suggest trying this first.
    
    2) I also noticed that sometimes maps would fill up again before the old memory was fully freed, leading to smaller spills. For example, for (Int, Int) pairs the first spill from 512 MB memory is about 68 MB of files, but later spills were only around 20 MB. I found that I could get better performance overall by adding some gc.collect() calls after every data.clear() and pdata.clear(). This freed more memory faster and allowed us to do more work in memory before spilling. The perf difference for one test job was around 30% but you should try it on your own jobs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15266679
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    +        return 0
    +
    +
    +class Merger(object):
    +    """
    +    merge shuffled data together by combinator
    +    """
    +
    +    def merge(self, iterator):
    +        raise NotImplementedError
    +
    +    def iteritems(self):
    +        raise NotImplementedError
    +
    +
    +class MapMerger(Merger):
    +    """
    +    In memory merger based on map
    +    """
    +
    +    def __init__(self, combiner):
    +        self.combiner = combiner
    +        self.data = {}
    +
    +    def merge(self, iterator):
    +        d, comb = self.data, self.combiner
    +        for k, v in iter(iterator):
    +            d[k] = comb(d[k], v) if k in d else v
    +
    +    def iteritems(self):
    +        return self.data.iteritems()
    +
    +
    +class ExternalHashMapMerger(Merger):
    +
    +    """
    +    External merger will dump the aggregated data into disks when memory usage
    +    is above the limit, then merge them together.
    +
    +    >>> combiner = lambda x, y:x+y
    +    >>> merger = ExternalHashMapMerger(combiner, 10)
    +    >>> N = 10000
    +    >>> merger.merge(zip(xrange(N), xrange(N)) * 10)
    +    >>> assert merger.spills > 0
    +    >>> sum(v for k,v in merger.iteritems())
    +    499950000
    +    """
    +
    +    PARTITIONS = 64
    +    BATCH = 10000
    +
    +    def __init__(self, combiner, memory_limit=512, serializer=None,
    +            localdirs=None, scale=1):
    +        self.combiner = combiner
    +        self.memory_limit = memory_limit
    +        self.serializer = serializer or\
    +                BatchedSerializer(AutoSerializer(), 1024)
    +        self.localdirs = localdirs or self._get_dirs()
    +        self.scale = scale
    +        self.data = {}
    +        self.pdata = []
    +        self.spills = 0
    +
    +    def _get_dirs(self):
    +        path = os.environ.get("SPARK_LOCAL_DIR", "/tmp/spark")
    +        dirs = path.split(",")
    +        localdirs = []
    +        for d in dirs:
    +            d = os.path.join(d, "merge", str(os.getpid()))
    +            try:
    +                os.makedirs(d)
    +                localdirs.append(d)
    +            except IOError:
    +                pass
    +        if not localdirs:
    +            raise IOError("no writable directories: " + path)
    +        return localdirs
    +
    +    def _get_spill_dir(self, n):
    +        return os.path.join(self.localdirs[n % len(self.localdirs)], str(n))
    +
    +    @property
    +    def used_memory(self):
    +        return get_used_memory()
    +
    +    @property
    +    def next_limit(self):
    +        return max(self.memory_limit, self.used_memory * 1.05)
    +
    +    def merge(self, iterator, check=True):
    +        """ merge (K,V) pair by combiner """
    +        iterator = iter(iterator)
    +        # speedup attribute lookup
    +        d, comb, batch = self.data, self.combiner, self.BATCH
    +        c = 0
    +        for k, v in iterator:
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not check:
    +                continue
    +
    +            c += 1
    +            if c % batch == 0 and self.used_memory > self.memory_limit:
    +                self._first_spill()
    +                self._partitioned_merge(iterator, self.next_limit)
    +                break
    +
    +    def _hash(self, key):
    +        return (hash(key) / self.scale) % self.PARTITIONS
    +
    +    def _partitioned_merge(self, iterator, limit):
    +        comb, pdata, hfun = self.combiner, self.pdata, self._hash
    +        c = 0
    +        for k, v in iterator:
    +            d = pdata[hfun(k)]
    +            d[k] = comb(d[k], v) if k in d else v
    +            if not limit:
    +                continue
    +            c += 1
    +            if c % self.BATCH == 0 and self.used_memory > limit:
    +                self._spill()
    +                limit = self.next_limit
    +
    +    def _first_spill(self):
    +        path = self._get_spill_dir(self.spills)
    +        if not os.path.exists(path):
    +            os.makedirs(path)
    +        streams = [open(os.path.join(path, str(i)), 'w')
    +                   for i in range(self.PARTITIONS)]
    +        for k, v in self.data.iteritems():
    +            h = self._hash(k)
    +            self.serializer.dump_stream([(k, v)], streams[h])
    +        for s in streams:
    +            s.close()
    +        self.data.clear()
    +        self.pdata = [{} for i in range(self.PARTITIONS)]
    +        self.spills += 1
    +
    +    def _spill(self):
    --- End diff --
    
    _first_spill() is only called for the first time, the data already be hold in memory, If we call _partitioned_merge() and _spill(), it will double the memory in worst case.
    
    I had add more docs for these two, does it make sense for you?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] spark pull request: [SPARK-2538] [PySpark] Hash based disk spillin...

Posted by mateiz <gi...@git.apache.org>.
Github user mateiz commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1460#discussion_r15263605
  
    --- Diff: python/pyspark/shuffle.py ---
    @@ -0,0 +1,258 @@
    +#
    +# 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 os
    +import sys
    +import platform
    +import shutil
    +import warnings
    +
    +from pyspark.serializers import BatchedSerializer, AutoSerializer
    +
    +try:
    +    import psutil
    +
    +    def get_used_memory():
    +        self = psutil.Process(os.getpid())
    +        return self.memory_info().rss >> 20
    +
    +except ImportError:
    +
    +    def get_used_memory():
    +        if platform.system() == 'Linux':
    +            for line in open('/proc/self/status'):
    +                if line.startswith('VmRSS:'):
    +                    return int(line.split()[1]) >> 10
    +        else:
    +            warnings.warn("please install psutil to get accurate memory usage")
    +            if platform.system() == "Darwin":
    +                import resource
    +                return resource.getrusage(resource.RUSAGE_SELF).ru_maxrss >> 20
    +            # TODO: support windows
    --- End diff --
    
    Alright, then you should open a JIRA saying that this is not supported on Windows, and maybe print a warning in spill() if the OS is Windows.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---