confused behavior about pyspark.sql, Row, schema, and createDataFrame

2015-12-23 Thread Chang Ya-Hsuan
python version: 2.7.9
os: ubuntu 14.04
spark: 1.5.2

```
import pyspark
from pyspark.sql import Row
from pyspark.sql.types import StructType, IntegerType
sc = pyspark.SparkContext()
sqlc = pyspark.SQLContext(sc)
schema1 = StructType() \
.add('a', IntegerType()) \
.add('b', IntegerType())

schema2 = StructType() \
.add('b', IntegerType()) \
.add('a', IntegerType())
print(schema1 == schema2)

r1 = Row(a=1, b=2)
r2 = Row(b=2, a=1)
print(r1 == r2)

data = [r1, r2]

df1 = sqlc.createDataFrame(data, schema1)
df1.show()

df2 = sqlc.createDataFrame(data, schema2)
df2.show()
```

intuitively, I thought df1 and df2 should contain the same data, however,
the output is

```
False
True
+---+---+
|  a|  b|
+---+---+
|  1|  2|
|  1|  2|
+---+---+
​
+---+---+
|  b|  a|
+---+---+
|  1|  2|
|  1|  2|
+---+---+
```

after trace the source code, I found

1. schema (StructType) use list store fields, so it is order-sensitive
https://github.com/apache/spark/blob/master/python/pyspark/sql/types.py#L459
2. Row will sort according to field name when new
https://github.com/apache/spark/blob/master/python/pyspark/sql/types.py#L1204
3. It seems (not 100% sure) that when createDataFrame, it access the filed
of Row by order not by filed name
https://github.com/apache/spark/blob/master/python/pyspark/sql/context.py#L422

This behavior is a little bit tricky. Maybe this behavior could be mention
at document?

Thanks.

-- 
-- 張雅軒


value of sc.defaultParallelism

2015-12-23 Thread Chang Ya-Hsuan
python version: 2.7.9
os: ubuntu 14.04
spark: 1.5.2

I run a standalone spark on localhost, and use the following code to access
sc.defaultParallism

# a.py
import pyspark
sc = pyspark.SparkContext()
print(sc.defaultParallelism)

and use the following command to submit

$  spark-submit --master spark://yahsuan-vm:7077 a.py

it prints 2, however, my spark web page shows I got 4 cores


​
according to http://spark.apache.org/docs/latest/configuration.html

spark.default.parallelismFor distributed shuffle operations likereduceByKey
 and join, the largest number of partitions in a parent RDD. For operations
likeparallelize with no parent RDDs, it depends on the cluster manager:

   - Local mode: number of cores on the local machine
   - Mesos fine grained mode: 8
   - Others: total number of cores on all executor nodes or 2, whichever is
   larger

Default number of partitions in RDDs returned by transformations like join,
reduceByKey, andparallelize when not set by user.
It seems I should get 4 rather than 2.
Am I misunderstood the document?

-- 
-- 張雅軒


Re: does spark really support label expr like && or || ?

2015-12-16 Thread Chang Ya-Hsuan
are you trying to do dataframe boolean expression?
please use '&' for 'and', '|' for 'or', '~' for 'not' when building
DataFrame boolean expressions.

example:

>>> df = sqlContext.range(10)
>>> df.where( (df.id==1) | ~(df.id==1))
DataFrame[id: bigint]


On Wed, Dec 16, 2015 at 4:32 PM, Allen Zhang  wrote:

> Hi All,
>
> does spark label expression really support "&&" or "||" or even "!" for
> label based schedulering?
> I tried that but it does NOT work.
>
> Best Regards,
> Allen
>
>


-- 
-- 張雅軒


Re: Failed to generate predicate Error when using dropna

2015-12-08 Thread Chang Ya-Hsuan
https://issues.apache.org/jira/browse/SPARK-12231

this is my first time to create JIRA ticket.
is this ticket proper?
thanks

On Tue, Dec 8, 2015 at 9:59 PM, Reynold Xin  wrote:

> Can you create a JIRA ticket for this? Thanks.
>
>
> On Tue, Dec 8, 2015 at 5:25 PM, Chang Ya-Hsuan  wrote:
>
>> spark version: spark-1.5.2-bin-hadoop2.6
>> python version: 2.7.9
>> os: ubuntu 14.04
>>
>> code to reproduce error
>>
>> # write.py
>>
>> import pyspark
>> sc = pyspark.SparkContext()
>> sqlc = pyspark.SQLContext(sc)
>> df = sqlc.range(10)
>> df1 = df.withColumn('a', df['id'] * 2)
>> df1.write.partitionBy('id').parquet('./data')
>>
>>
>> # read.py
>>
>> import pyspark
>> sc = pyspark.SparkContext()
>> sqlc = pyspark.SQLContext(sc)
>> df2 = sqlc.read.parquet('./data')
>> df2.dropna().count()
>>
>>
>> $ spark-submit write.py
>> $ spark-submit read.py
>>
>> # error message
>>
>> 15/12/08 17:20:34 ERROR Filter: Failed to generate predicate, fallback to
>> interpreted org.apache.spark.sql.catalyst.errors.package$TreeNodeException:
>> Binding attribute, tree: a#0L
>> ...
>>
>> If write data without partitionBy, the error won't happen
>> any suggestion?
>> Thanks!
>>
>> --
>> -- 張雅軒
>>
>
>


-- 
-- 張雅軒


Failed to generate predicate Error when using dropna

2015-12-08 Thread Chang Ya-Hsuan
spark version: spark-1.5.2-bin-hadoop2.6
python version: 2.7.9
os: ubuntu 14.04

code to reproduce error

# write.py

import pyspark
sc = pyspark.SparkContext()
sqlc = pyspark.SQLContext(sc)
df = sqlc.range(10)
df1 = df.withColumn('a', df['id'] * 2)
df1.write.partitionBy('id').parquet('./data')


# read.py

import pyspark
sc = pyspark.SparkContext()
sqlc = pyspark.SQLContext(sc)
df2 = sqlc.read.parquet('./data')
df2.dropna().count()


$ spark-submit write.py
$ spark-submit read.py

# error message

15/12/08 17:20:34 ERROR Filter: Failed to generate predicate, fallback to
interpreted org.apache.spark.sql.catalyst.errors.package$TreeNodeException:
Binding attribute, tree: a#0L
...

If write data without partitionBy, the error won't happen
any suggestion?
Thanks!

-- 
-- 張雅軒


Re: pyspark with pypy not work for spark-1.5.1

2015-11-06 Thread Chang Ya-Hsuan
Hi I run ./python/ru-tests to test following modules of spark-1.5.1:

[pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql',
'pyspark-streaming]

against to following pypy versions:

pypy-2.2.1  pypy-2.3  pypy-2.3.1  pypy-2.4.0  pypy-2.5.0  pypy-2.5.1
 pypy-2.6.0  pypy-2.6.1  pypy-4.0.0

except pypy-2.2.1, all others pass the test.

the error message of pypy-2.2.1 is:

Traceback (most recent call last):
  File "app_main.py", line 72, in run_toplevel
  File "/home/yahsuan/.pyenv/versions/pypy-2.2.1/lib-python/2.7/runpy.py",
line 151, in _run_module_as_main
mod_name, loader, code, fname = _get_module_details(mod_name)
  File "/home/yahsuan/.pyenv/versions/pypy-2.2.1/lib-python/2.7/runpy.py",
line 101, in _get_module_details
loader = get_loader(mod_name)
  File
"/home/yahsuan/.pyenv/versions/pypy-2.2.1/lib-python/2.7/pkgutil.py", line
465, in get_loader
return find_loader(fullname)
  File
"/home/yahsuan/.pyenv/versions/pypy-2.2.1/lib-python/2.7/pkgutil.py", line
475, in find_loader
for importer in iter_importers(fullname):
  File
"/home/yahsuan/.pyenv/versions/pypy-2.2.1/lib-python/2.7/pkgutil.py", line
431, in iter_importers
__import__(pkg)
  File "pyspark/__init__.py", line 41, in 
from pyspark.context import SparkContext
  File "pyspark/context.py", line 26, in 
from pyspark import accumulators
  File "pyspark/accumulators.py", line 98, in 
from pyspark.serializers import read_int, PickleSerializer
  File "pyspark/serializers.py", line 400, in 
_hijack_namedtuple()
  File "pyspark/serializers.py", line 378, in _hijack_namedtuple
_old_namedtuple = _copy_func(collections.namedtuple)
  File "pyspark/serializers.py", line 376, in _copy_func
f.__defaults__, f.__closure__)
AttributeError: 'function' object has no attribute '__closure__'

p.s. would you want to test different pypy versions on your Jenkins? maybe
I could help

On Fri, Nov 6, 2015 at 2:23 AM, Josh Rosen  wrote:

> You could try running PySpark's own unit tests. Try ./python/run-tests
> --help for instructions.
>
> On Thu, Nov 5, 2015 at 12:31 AM Chang Ya-Hsuan  wrote:
>
>> I've test on following pypy version against to spark-1.5.1
>>
>>   pypy-2.2.1
>>   pypy-2.3
>>   pypy-2.3.1
>>   pypy-2.4.0
>>   pypy-2.5.0
>>   pypy-2.5.1
>>   pypy-2.6.0
>>   pypy-2.6.1
>>
>> I run
>>
>> $ PYSPARK_PYTHON=/path/to/pypy-xx.xx/bin/pypy
>> /path/to/spark-1.5.1/bin/pyspark
>>
>> and only pypy-2.2.1 failed.
>>
>> Any suggestion to run advanced test?
>>
>> On Thu, Nov 5, 2015 at 4:14 PM, Chang Ya-Hsuan 
>> wrote:
>>
>>> Thanks for your quickly reply.
>>>
>>> I will test several pypy versions and report the result later.
>>>
>>> On Thu, Nov 5, 2015 at 4:06 PM, Josh Rosen  wrote:
>>>
>>>> I noticed that you're using PyPy 2.2.1, but it looks like Spark 1.5.1's
>>>> docs say that we only support PyPy 2.3+. Could you try using a newer PyPy
>>>> version to see if that works?
>>>>
>>>> I just checked and it looks like our Jenkins tests are running against
>>>> PyPy 2.5.1, so that version is known to work. I'm not sure what the actual
>>>> minimum supported PyPy version is. Would you be interested in helping to
>>>> investigate so that we can update the documentation or produce a fix to
>>>> restore compatibility with earlier PyPy builds?
>>>>
>>>> On Wed, Nov 4, 2015 at 11:56 PM, Chang Ya-Hsuan 
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I am trying to run pyspark with pypy, and it is work when using
>>>>> spark-1.3.1 but failed when using spark-1.4.1 and spark-1.5.1
>>>>>
>>>>> my pypy version:
>>>>>
>>>>> $ /usr/bin/pypy --version
>>>>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>>>>> [PyPy 2.2.1 with GCC 4.8.4]
>>>>>
>>>>> works with spark-1.3.1
>>>>>
>>>>> $ PYSPARK_PYTHON=/usr/bin/pypy
>>>>> ~/Tool/spark-1.3.1-bin-hadoop2.6/bin/pyspark
>>>>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>>>>> [PyPy 2.2.1 with GCC 4.8.4] on linux2
>>>>> Type "help", "copyright", "credits" or "license" for more information.
>>>>> 15/11/05 15:50:30 WARN Utils: Your hostname, xx resolves to a
>>>>> loopback 

Re: pyspark with pypy not work for spark-1.5.1

2015-11-05 Thread Chang Ya-Hsuan
I've test on following pypy version against to spark-1.5.1

  pypy-2.2.1
  pypy-2.3
  pypy-2.3.1
  pypy-2.4.0
  pypy-2.5.0
  pypy-2.5.1
  pypy-2.6.0
  pypy-2.6.1

I run

$ PYSPARK_PYTHON=/path/to/pypy-xx.xx/bin/pypy
/path/to/spark-1.5.1/bin/pyspark

and only pypy-2.2.1 failed.

Any suggestion to run advanced test?

On Thu, Nov 5, 2015 at 4:14 PM, Chang Ya-Hsuan  wrote:

> Thanks for your quickly reply.
>
> I will test several pypy versions and report the result later.
>
> On Thu, Nov 5, 2015 at 4:06 PM, Josh Rosen  wrote:
>
>> I noticed that you're using PyPy 2.2.1, but it looks like Spark 1.5.1's
>> docs say that we only support PyPy 2.3+. Could you try using a newer PyPy
>> version to see if that works?
>>
>> I just checked and it looks like our Jenkins tests are running against
>> PyPy 2.5.1, so that version is known to work. I'm not sure what the actual
>> minimum supported PyPy version is. Would you be interested in helping to
>> investigate so that we can update the documentation or produce a fix to
>> restore compatibility with earlier PyPy builds?
>>
>> On Wed, Nov 4, 2015 at 11:56 PM, Chang Ya-Hsuan 
>> wrote:
>>
>>> Hi all,
>>>
>>> I am trying to run pyspark with pypy, and it is work when using
>>> spark-1.3.1 but failed when using spark-1.4.1 and spark-1.5.1
>>>
>>> my pypy version:
>>>
>>> $ /usr/bin/pypy --version
>>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>>> [PyPy 2.2.1 with GCC 4.8.4]
>>>
>>> works with spark-1.3.1
>>>
>>> $ PYSPARK_PYTHON=/usr/bin/pypy
>>> ~/Tool/spark-1.3.1-bin-hadoop2.6/bin/pyspark
>>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>>> [PyPy 2.2.1 with GCC 4.8.4] on linux2
>>> Type "help", "copyright", "credits" or "license" for more information.
>>> 15/11/05 15:50:30 WARN Utils: Your hostname, xx resolves to a
>>> loopback address: 127.0.1.1; using xxx.xxx.xxx.xxx instead (on interface
>>> eth0)
>>> 15/11/05 15:50:30 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to
>>> another address
>>> 15/11/05 15:50:31 WARN NativeCodeLoader: Unable to load native-hadoop
>>> library for your platform... using builtin-java classes where applicable
>>> Welcome to
>>>     __
>>>  / __/__  ___ _/ /__
>>> _\ \/ _ \/ _ `/ __/  '_/
>>>/__ / .__/\_,_/_/ /_/\_\   version 1.3.1
>>>   /_/
>>>
>>> Using Python version 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015)
>>> SparkContext available as sc, HiveContext available as sqlContext.
>>> And now for something completely different: ``Armin: "Prolog is a
>>> mess.", CF:
>>> "No, it's very cool!", Armin: "Isn't this what I said?"''
>>> >>>
>>>
>>> error message for 1.5.1
>>>
>>> $ PYSPARK_PYTHON=/usr/bin/pypy
>>> ~/Tool/spark-1.5.1-bin-hadoop2.6/bin/pyspark
>>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>>> [PyPy 2.2.1 with GCC 4.8.4] on linux2
>>> Type "help", "copyright", "credits" or "license" for more information.
>>> Traceback (most recent call last):
>>>   File "app_main.py", line 72, in run_toplevel
>>>   File "app_main.py", line 614, in run_it
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/shell.py",
>>> line 30, in 
>>> import pyspark
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/__init__.py",
>>> line 41, in 
>>> from pyspark.context import SparkContext
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/context.py",
>>> line 26, in 
>>> from pyspark import accumulators
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/accumulators.py",
>>> line 98, in 
>>> from pyspark.serializers import read_int, PickleSerializer
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>>> line 400, in 
>>> _hijack_namedtuple()
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>>> line 378, in _hijack_namedtuple
>>> _old_namedtuple = _copy_func(collections.namedtuple)
>>>   File
>>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>>> line 376, in _copy_func
>>> f.__defaults__, f.__closure__)
>>> AttributeError: 'function' object has no attribute '__closure__'
>>> And now for something completely different: ``the traces don't lie''
>>>
>>> is this a known issue? any suggestion to resolve it? or how can I help
>>> to fix this problem?
>>>
>>> Thanks.
>>>
>>
>>
>
>
> --
> -- 張雅軒
>



-- 
-- 張雅軒


Re: pyspark with pypy not work for spark-1.5.1

2015-11-05 Thread Chang Ya-Hsuan
Thanks for your quickly reply.

I will test several pypy versions and report the result later.

On Thu, Nov 5, 2015 at 4:06 PM, Josh Rosen  wrote:

> I noticed that you're using PyPy 2.2.1, but it looks like Spark 1.5.1's
> docs say that we only support PyPy 2.3+. Could you try using a newer PyPy
> version to see if that works?
>
> I just checked and it looks like our Jenkins tests are running against
> PyPy 2.5.1, so that version is known to work. I'm not sure what the actual
> minimum supported PyPy version is. Would you be interested in helping to
> investigate so that we can update the documentation or produce a fix to
> restore compatibility with earlier PyPy builds?
>
> On Wed, Nov 4, 2015 at 11:56 PM, Chang Ya-Hsuan 
> wrote:
>
>> Hi all,
>>
>> I am trying to run pyspark with pypy, and it is work when using
>> spark-1.3.1 but failed when using spark-1.4.1 and spark-1.5.1
>>
>> my pypy version:
>>
>> $ /usr/bin/pypy --version
>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>> [PyPy 2.2.1 with GCC 4.8.4]
>>
>> works with spark-1.3.1
>>
>> $ PYSPARK_PYTHON=/usr/bin/pypy
>> ~/Tool/spark-1.3.1-bin-hadoop2.6/bin/pyspark
>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>> [PyPy 2.2.1 with GCC 4.8.4] on linux2
>> Type "help", "copyright", "credits" or "license" for more information.
>> 15/11/05 15:50:30 WARN Utils: Your hostname, xx resolves to a
>> loopback address: 127.0.1.1; using xxx.xxx.xxx.xxx instead (on interface
>> eth0)
>> 15/11/05 15:50:30 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to
>> another address
>> 15/11/05 15:50:31 WARN NativeCodeLoader: Unable to load native-hadoop
>> library for your platform... using builtin-java classes where applicable
>> Welcome to
>>     __
>>  / __/__  ___ _/ /__
>> _\ \/ _ \/ _ `/ __/  '_/
>>/__ / .__/\_,_/_/ /_/\_\   version 1.3.1
>>   /_/
>>
>> Using Python version 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015)
>> SparkContext available as sc, HiveContext available as sqlContext.
>> And now for something completely different: ``Armin: "Prolog is a mess.",
>> CF:
>> "No, it's very cool!", Armin: "Isn't this what I said?"''
>> >>>
>>
>> error message for 1.5.1
>>
>> $ PYSPARK_PYTHON=/usr/bin/pypy
>> ~/Tool/spark-1.5.1-bin-hadoop2.6/bin/pyspark
>> Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
>> [PyPy 2.2.1 with GCC 4.8.4] on linux2
>> Type "help", "copyright", "credits" or "license" for more information.
>> Traceback (most recent call last):
>>   File "app_main.py", line 72, in run_toplevel
>>   File "app_main.py", line 614, in run_it
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/shell.py",
>> line 30, in 
>> import pyspark
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/__init__.py",
>> line 41, in 
>> from pyspark.context import SparkContext
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/context.py",
>> line 26, in 
>> from pyspark import accumulators
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/accumulators.py",
>> line 98, in 
>> from pyspark.serializers import read_int, PickleSerializer
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>> line 400, in 
>> _hijack_namedtuple()
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>> line 378, in _hijack_namedtuple
>> _old_namedtuple = _copy_func(collections.namedtuple)
>>   File
>> "/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
>> line 376, in _copy_func
>> f.__defaults__, f.__closure__)
>> AttributeError: 'function' object has no attribute '__closure__'
>> And now for something completely different: ``the traces don't lie''
>>
>> is this a known issue? any suggestion to resolve it? or how can I help to
>> fix this problem?
>>
>> Thanks.
>>
>
>


-- 
-- 張雅軒


pyspark with pypy not work for spark-1.5.1

2015-11-04 Thread Chang Ya-Hsuan
Hi all,

I am trying to run pyspark with pypy, and it is work when using spark-1.3.1
but failed when using spark-1.4.1 and spark-1.5.1

my pypy version:

$ /usr/bin/pypy --version
Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
[PyPy 2.2.1 with GCC 4.8.4]

works with spark-1.3.1

$ PYSPARK_PYTHON=/usr/bin/pypy ~/Tool/spark-1.3.1-bin-hadoop2.6/bin/pyspark
Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
[PyPy 2.2.1 with GCC 4.8.4] on linux2
Type "help", "copyright", "credits" or "license" for more information.
15/11/05 15:50:30 WARN Utils: Your hostname, xx resolves to a loopback
address: 127.0.1.1; using xxx.xxx.xxx.xxx instead (on interface eth0)
15/11/05 15:50:30 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to
another address
15/11/05 15:50:31 WARN NativeCodeLoader: Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable
Welcome to
    __
 / __/__  ___ _/ /__
_\ \/ _ \/ _ `/ __/  '_/
   /__ / .__/\_,_/_/ /_/\_\   version 1.3.1
  /_/

Using Python version 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015)
SparkContext available as sc, HiveContext available as sqlContext.
And now for something completely different: ``Armin: "Prolog is a mess.",
CF:
"No, it's very cool!", Armin: "Isn't this what I said?"''
>>>

error message for 1.5.1

$ PYSPARK_PYTHON=/usr/bin/pypy ~/Tool/spark-1.5.1-bin-hadoop2.6/bin/pyspark
Python 2.7.3 (2.2.1+dfsg-1ubuntu0.3, Sep 30 2015, 15:18:40)
[PyPy 2.2.1 with GCC 4.8.4] on linux2
Type "help", "copyright", "credits" or "license" for more information.
Traceback (most recent call last):
  File "app_main.py", line 72, in run_toplevel
  File "app_main.py", line 614, in run_it
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/shell.py",
line 30, in 
import pyspark
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/__init__.py",
line 41, in 
from pyspark.context import SparkContext
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/context.py",
line 26, in 
from pyspark import accumulators
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/accumulators.py",
line 98, in 
from pyspark.serializers import read_int, PickleSerializer
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
line 400, in 
_hijack_namedtuple()
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
line 378, in _hijack_namedtuple
_old_namedtuple = _copy_func(collections.namedtuple)
  File
"/home/yahsuan/Tool/spark-1.5.1-bin-hadoop2.6/python/pyspark/serializers.py",
line 376, in _copy_func
f.__defaults__, f.__closure__)
AttributeError: 'function' object has no attribute '__closure__'
And now for something completely different: ``the traces don't lie''

is this a known issue? any suggestion to resolve it? or how can I help to
fix this problem?

Thanks.