Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-8365

pyspark does not retain --packages or --jars passed on the command line as of 1.4.0

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Closed
    • Blocker
    • Resolution: Duplicate
    • 1.4.0
    • None
    • PySpark, SQL
    • None

    Description

      I downloaded the pre-compiled Spark 1.4.0 and attempted to run an existing Python Spark application against it and got the following error:

      py4j.protocol.Py4JJavaError: An error occurred while calling o90.save.
      : java.lang.RuntimeException: Failed to load class for data source: com.databricks.spark.csv

      I pass the following on the command-line to my spark-submit:
      --packages com.databricks:spark-csv_2.10:1.0.3

      This worked fine on 1.3.1, but not in 1.4.

      I was able to replicate it with the following pyspark:

      a = {'a':1.0, 'b':'asdf'}
      rdd = sc.parallelize([a])
      df = sqlContext.createDataFrame(rdd)
      df.save("/tmp/d.csv", "com.databricks.spark.csv")
      

      Even using the new df.write.format('com.databricks.spark.csv').save('/tmp/d.csv') gives the same error.

      I see it was added in the web UI:
      file:/Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar Added By User
      file:/Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar Added By User
      http://10.0.0.222:56871/jars/com.databricks_spark-csv_2.10-1.0.3.jar Added By User
      http://10.0.0.222:56871/jars/org.apache.commons_commons-csv-1.1.jar Added By User

      Thoughts?

      I also attempted using the Scala spark-shell to load a csv using the same package and it worked just fine, so this seems specific to pyspark.

      -Don

      Gory details:

      $ pyspark --packages "com.databricks:spark-csv_2.10:1.0.3"
      Python 2.7.6 (default, Sep  9 2014, 15:04:36)
      [GCC 4.2.1 Compatible Apple LLVM 6.0 (clang-600.0.39)] on darwin
      Type "help", "copyright", "credits" or "license" for more information.
      Ivy Default Cache set to: /Users/drake/.ivy2/cache
      The jars for the packages stored in: /Users/drake/.ivy2/jars
      :: loading settings :: url = jar:file:/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/lib/spark-assembly-1.4.0-hadoop2.6.0.jar!/org/apache/ivy/core/settings/ivysettings.xml
      com.databricks#spark-csv_2.10 added as a dependency
      :: resolving dependencies :: org.apache.spark#spark-submit-parent;1.0
      	confs: [default]
      	found com.databricks#spark-csv_2.10;1.0.3 in central
      	found org.apache.commons#commons-csv;1.1 in central
      :: resolution report :: resolve 590ms :: artifacts dl 17ms
      	:: modules in use:
      	com.databricks#spark-csv_2.10;1.0.3 from central in [default]
      	org.apache.commons#commons-csv;1.1 from central in [default]
      	---------------------------------------------------------------------
      	|                  |            modules            ||   artifacts   |
      	|       conf       | number| search|dwnlded|evicted|| number|dwnlded|
      	---------------------------------------------------------------------
      	|      default     |   2   |   0   |   0   |   0   ||   2   |   0   |
      	---------------------------------------------------------------------
      :: retrieving :: org.apache.spark#spark-submit-parent
      	confs: [default]
      	0 artifacts copied, 2 already retrieved (0kB/15ms)
      Using Spark's default log4j profile: org/apache/spark/log4j-defaults.properties
      15/06/13 11:06:08 INFO SparkContext: Running Spark version 1.4.0
      2015-06-13 11:06:08.921 java[19233:2145789] Unable to load realm info from SCDynamicStore
      15/06/13 11:06:09 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
      15/06/13 11:06:09 WARN Utils: Your hostname, Dons-MacBook-Pro-2.local resolves to a loopback address: 127.0.0.1; using 10.0.0.222 instead (on interface en0)
      15/06/13 11:06:09 WARN Utils: Set SPARK_LOCAL_IP if you need to bind to another address
      15/06/13 11:06:09 INFO SecurityManager: Changing view acls to: drake
      15/06/13 11:06:09 INFO SecurityManager: Changing modify acls to: drake
      15/06/13 11:06:09 INFO SecurityManager: SecurityManager: authentication disabled; ui acls disabled; users with view permissions: Set(drake); users with modify permissions: Set(drake)
      15/06/13 11:06:10 INFO Slf4jLogger: Slf4jLogger started
      15/06/13 11:06:10 INFO Remoting: Starting remoting
      15/06/13 11:06:10 INFO Remoting: Remoting started; listening on addresses :[akka.tcp://sparkDriver@10.0.0.222:56870]
      15/06/13 11:06:10 INFO Utils: Successfully started service 'sparkDriver' on port 56870.
      15/06/13 11:06:10 INFO SparkEnv: Registering MapOutputTracker
      15/06/13 11:06:10 INFO SparkEnv: Registering BlockManagerMaster
      15/06/13 11:06:10 INFO DiskBlockManager: Created local directory at /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/blockmgr-a1412b71-fe56-429c-a193-ce3fb95d2ffd
      15/06/13 11:06:10 INFO MemoryStore: MemoryStore started with capacity 265.4 MB
      15/06/13 11:06:10 INFO HttpFileServer: HTTP File server directory is /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/httpd-84d178da-7e60-4eed-8031-e6a0c465bd4c
      15/06/13 11:06:10 INFO HttpServer: Starting HTTP Server
      15/06/13 11:06:10 INFO Utils: Successfully started service 'HTTP file server' on port 56871.
      15/06/13 11:06:10 INFO SparkEnv: Registering OutputCommitCoordinator
      15/06/13 11:06:11 WARN Utils: Service 'SparkUI' could not bind on port 4040. Attempting port 4041.
      15/06/13 11:06:11 INFO Utils: Successfully started service 'SparkUI' on port 4041.
      15/06/13 11:06:11 INFO SparkUI: Started SparkUI at http://10.0.0.222:4041
      15/06/13 11:06:11 INFO SparkContext: Added JAR file:/Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar at http://10.0.0.222:56871/jars/com.databricks_spark-csv_2.10-1.0.3.jar with timestamp 1434211571303
      15/06/13 11:06:11 INFO SparkContext: Added JAR file:/Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar at http://10.0.0.222:56871/jars/org.apache.commons_commons-csv-1.1.jar with timestamp 1434211571326
      15/06/13 11:06:11 INFO Utils: Copying /Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/com.databricks_spark-csv_2.10-1.0.3.jar
      15/06/13 11:06:11 INFO SparkContext: Added file file:/Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar at file:/Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar with timestamp 1434211571468
      15/06/13 11:06:11 INFO Utils: Copying /Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/org.apache.commons_commons-csv-1.1.jar
      15/06/13 11:06:11 INFO SparkContext: Added file file:/Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar at file:/Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar with timestamp 1434211571502
      15/06/13 11:06:11 INFO Executor: Starting executor ID driver on host localhost
      15/06/13 11:06:11 INFO Utils: Successfully started service 'org.apache.spark.network.netty.NettyBlockTransferService' on port 56872.
      15/06/13 11:06:11 INFO NettyBlockTransferService: Server created on 56872
      15/06/13 11:06:11 INFO BlockManagerMaster: Trying to register BlockManager
      15/06/13 11:06:11 INFO BlockManagerMasterEndpoint: Registering block manager localhost:56872 with 265.4 MB RAM, BlockManagerId(driver, localhost, 56872)
      15/06/13 11:06:11 INFO BlockManagerMaster: Registered BlockManager
      Welcome to
            ____              __
           / __/__  ___ _____/ /__
          _\ \/ _ \/ _ `/ __/  '_/
         /__ / .__/\_,_/_/ /_/\_\   version 1.4.0
            /_/
      
      Using Python version 2.7.6 (default, Sep  9 2014 15:04:36)
      SparkContext available as sc, HiveContext available as sqlContext.
      >>> a = {'a':1.0, 'b':'asdf'}
      >>> rdd = sc.parallelize([a])
      >>> df = sqlContext.createDataFrame(rdd)
      15/06/13 11:06:50 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:50 INFO DAGScheduler: Got job 0 (runJob at PythonRDD.scala:366) with 1 output partitions (allowLocal=true)
      15/06/13 11:06:50 INFO DAGScheduler: Final stage: ResultStage 0(runJob at PythonRDD.scala:366)
      15/06/13 11:06:50 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:50 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:50 INFO DAGScheduler: Submitting ResultStage 0 (PythonRDD[1] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:51 INFO MemoryStore: ensureFreeSpace(3672) called with curMem=0, maxMem=278302556
      15/06/13 11:06:51 INFO MemoryStore: Block broadcast_0 stored as values in memory (estimated size 3.6 KB, free 265.4 MB)
      15/06/13 11:06:51 INFO MemoryStore: ensureFreeSpace(2328) called with curMem=3672, maxMem=278302556
      15/06/13 11:06:51 INFO MemoryStore: Block broadcast_0_piece0 stored as bytes in memory (estimated size 2.3 KB, free 265.4 MB)
      15/06/13 11:06:51 INFO BlockManagerInfo: Added broadcast_0_piece0 in memory on localhost:56872 (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:06:51 INFO SparkContext: Created broadcast 0 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:51 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 0 (PythonRDD[1] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:51 INFO TaskSchedulerImpl: Adding task set 0.0 with 1 tasks
      15/06/13 11:06:51 INFO TaskSetManager: Starting task 0.0 in stage 0.0 (TID 0, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:51 INFO Executor: Running task 0.0 in stage 0.0 (TID 0)
      15/06/13 11:06:51 INFO Executor: Fetching file:/Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar with timestamp 1434211571502
      15/06/13 11:06:51 INFO Utils: /Users/drake/.ivy2/jars/org.apache.commons_commons-csv-1.1.jar has been previously copied to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/org.apache.commons_commons-csv-1.1.jar
      15/06/13 11:06:51 INFO Executor: Fetching file:/Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar with timestamp 1434211571468
      15/06/13 11:06:51 INFO Utils: /Users/drake/.ivy2/jars/com.databricks_spark-csv_2.10-1.0.3.jar has been previously copied to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/com.databricks_spark-csv_2.10-1.0.3.jar
      15/06/13 11:06:51 INFO Executor: Fetching http://10.0.0.222:56871/jars/org.apache.commons_commons-csv-1.1.jar with timestamp 1434211571326
      15/06/13 11:06:51 INFO Utils: Fetching http://10.0.0.222:56871/jars/org.apache.commons_commons-csv-1.1.jar to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/fetchFileTemp2449082240048543653.tmp
      15/06/13 11:06:51 INFO Utils: /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/fetchFileTemp2449082240048543653.tmp has been previously copied to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/org.apache.commons_commons-csv-1.1.jar
      15/06/13 11:06:51 INFO Executor: Adding file:/private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/org.apache.commons_commons-csv-1.1.jar to class loader
      15/06/13 11:06:51 INFO Executor: Fetching http://10.0.0.222:56871/jars/com.databricks_spark-csv_2.10-1.0.3.jar with timestamp 1434211571303
      15/06/13 11:06:51 INFO Utils: Fetching http://10.0.0.222:56871/jars/com.databricks_spark-csv_2.10-1.0.3.jar to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/fetchFileTemp1396931258018379545.tmp
      15/06/13 11:06:51 INFO Utils: /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/fetchFileTemp1396931258018379545.tmp has been previously copied to /private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/com.databricks_spark-csv_2.10-1.0.3.jar
      15/06/13 11:06:51 INFO Executor: Adding file:/private/var/folders/7_/k5h82ws97b95v5f5h8wf9j0h0000gn/T/spark-f36f39f5-7f82-42e0-b3e0-9eb1e1cc0816/userFiles-1cab505b-7e88-4f9c-82d9-d6b361689d9d/com.databricks_spark-csv_2.10-1.0.3.jar to class loader
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 3165, boot = 3155, init = 10, finish = 0
      15/06/13 11:06:54 INFO Executor: Finished task 0.0 in stage 0.0 (TID 0). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 0.0 in stage 0.0 (TID 0) in 3505 ms on localhost (1/1)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Removed TaskSet 0.0, whose tasks have all completed, from pool
      15/06/13 11:06:54 INFO DAGScheduler: ResultStage 0 (runJob at PythonRDD.scala:366) finished in 3.525 s
      15/06/13 11:06:54 INFO DAGScheduler: Job 0 finished: runJob at PythonRDD.scala:366, took 3.852112 s
      15/06/13 11:06:54 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:54 INFO DAGScheduler: Got job 1 (runJob at PythonRDD.scala:366) with 4 output partitions (allowLocal=true)
      15/06/13 11:06:54 INFO DAGScheduler: Final stage: ResultStage 1(runJob at PythonRDD.scala:366)
      15/06/13 11:06:54 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:54 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:54 INFO DAGScheduler: Submitting ResultStage 1 (PythonRDD[2] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(3672) called with curMem=6000, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_1 stored as values in memory (estimated size 3.6 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(2330) called with curMem=9672, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_1_piece0 stored as bytes in memory (estimated size 2.3 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO BlockManagerInfo: Added broadcast_1_piece0 in memory on localhost:56872 (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:06:54 INFO SparkContext: Created broadcast 1 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:54 INFO DAGScheduler: Submitting 4 missing tasks from ResultStage 1 (PythonRDD[2] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Adding task set 1.0 with 4 tasks
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 0.0 in stage 1.0 (TID 1, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 1.0 in stage 1.0 (TID 2, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 2.0 in stage 1.0 (TID 3, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 3.0 in stage 1.0 (TID 4, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO Executor: Running task 0.0 in stage 1.0 (TID 1)
      15/06/13 11:06:54 INFO Executor: Running task 1.0 in stage 1.0 (TID 2)
      15/06/13 11:06:54 INFO Executor: Running task 2.0 in stage 1.0 (TID 3)
      15/06/13 11:06:54 INFO Executor: Running task 3.0 in stage 1.0 (TID 4)
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 2, boot = -15, init = 17, finish = 0
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 9, boot = 6, init = 2, finish = 1
      15/06/13 11:06:54 INFO Executor: Finished task 3.0 in stage 1.0 (TID 4). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO Executor: Finished task 0.0 in stage 1.0 (TID 1). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 13, boot = 9, init = 4, finish = 0
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 3.0 in stage 1.0 (TID 4) in 24 ms on localhost (1/4)
      15/06/13 11:06:54 INFO Executor: Finished task 2.0 in stage 1.0 (TID 3). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 0.0 in stage 1.0 (TID 1) in 28 ms on localhost (2/4)
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 2.0 in stage 1.0 (TID 3) in 27 ms on localhost (3/4)
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 28, boot = 28, init = 0, finish = 0
      15/06/13 11:06:54 INFO Executor: Finished task 1.0 in stage 1.0 (TID 2). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 1.0 in stage 1.0 (TID 2) in 42 ms on localhost (4/4)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Removed TaskSet 1.0, whose tasks have all completed, from pool
      15/06/13 11:06:54 INFO DAGScheduler: ResultStage 1 (runJob at PythonRDD.scala:366) finished in 0.044 s
      15/06/13 11:06:54 INFO DAGScheduler: Job 1 finished: runJob at PythonRDD.scala:366, took 0.063304 s
      15/06/13 11:06:54 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:54 INFO DAGScheduler: Got job 2 (runJob at PythonRDD.scala:366) with 3 output partitions (allowLocal=true)
      15/06/13 11:06:54 INFO DAGScheduler: Final stage: ResultStage 2(runJob at PythonRDD.scala:366)
      15/06/13 11:06:54 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:54 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:54 INFO DAGScheduler: Submitting ResultStage 2 (PythonRDD[3] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(3672) called with curMem=12002, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_2 stored as values in memory (estimated size 3.6 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(2330) called with curMem=15674, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_2_piece0 stored as bytes in memory (estimated size 2.3 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO BlockManagerInfo: Added broadcast_2_piece0 in memory on localhost:56872 (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:06:54 INFO SparkContext: Created broadcast 2 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:54 INFO DAGScheduler: Submitting 3 missing tasks from ResultStage 2 (PythonRDD[3] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Adding task set 2.0 with 3 tasks
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 0.0 in stage 2.0 (TID 5, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 1.0 in stage 2.0 (TID 6, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 2.0 in stage 2.0 (TID 7, localhost, PROCESS_LOCAL, 1708 bytes)
      15/06/13 11:06:54 INFO Executor: Running task 0.0 in stage 2.0 (TID 5)
      15/06/13 11:06:54 INFO Executor: Running task 1.0 in stage 2.0 (TID 6)
      15/06/13 11:06:54 INFO Executor: Running task 2.0 in stage 2.0 (TID 7)
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 2, boot = -41, init = 43, finish = 0
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 2, boot = -38, init = 40, finish = 0
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 1, boot = -77, init = 78, finish = 0
      15/06/13 11:06:54 INFO Executor: Finished task 0.0 in stage 2.0 (TID 5). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO Executor: Finished task 1.0 in stage 2.0 (TID 6). 666 bytes result sent to driver
      15/06/13 11:06:54 INFO Executor: Finished task 2.0 in stage 2.0 (TID 7). 722 bytes result sent to driver
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 0.0 in stage 2.0 (TID 5) in 14 ms on localhost (1/3)
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 1.0 in stage 2.0 (TID 6) in 13 ms on localhost (2/3)
      15/06/13 11:06:54 INFO TaskSetManager: Finished task 2.0 in stage 2.0 (TID 7) in 13 ms on localhost (3/3)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Removed TaskSet 2.0, whose tasks have all completed, from pool
      15/06/13 11:06:54 INFO DAGScheduler: ResultStage 2 (runJob at PythonRDD.scala:366) finished in 0.016 s
      15/06/13 11:06:54 INFO DAGScheduler: Job 2 finished: runJob at PythonRDD.scala:366, took 0.087522 s
      /Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/pyspark/sql/context.py:198: UserWarning: Using RDD of dict to inferSchema is deprecated,please use pyspark.sql.Row instead
        warnings.warn("Using RDD of dict to inferSchema is deprecated,"
      15/06/13 11:06:54 INFO BlockManagerInfo: Removed broadcast_1_piece0 on localhost:56872 in memory (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:06:54 INFO BlockManagerInfo: Removed broadcast_0_piece0 on localhost:56872 in memory (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:06:54 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:54 INFO DAGScheduler: Got job 3 (runJob at PythonRDD.scala:366) with 1 output partitions (allowLocal=true)
      15/06/13 11:06:54 INFO DAGScheduler: Final stage: ResultStage 3(runJob at PythonRDD.scala:366)
      15/06/13 11:06:54 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:54 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:54 INFO DAGScheduler: Submitting ResultStage 3 (PythonRDD[4] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(5120) called with curMem=6002, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_3 stored as values in memory (estimated size 5.0 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO MemoryStore: ensureFreeSpace(3338) called with curMem=11122, maxMem=278302556
      15/06/13 11:06:54 INFO MemoryStore: Block broadcast_3_piece0 stored as bytes in memory (estimated size 3.3 KB, free 265.4 MB)
      15/06/13 11:06:54 INFO BlockManagerInfo: Added broadcast_3_piece0 in memory on localhost:56872 (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:54 INFO SparkContext: Created broadcast 3 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:54 INFO DAGScheduler: Submitting 1 missing tasks from ResultStage 3 (PythonRDD[4] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:54 INFO TaskSchedulerImpl: Adding task set 3.0 with 1 tasks
      15/06/13 11:06:54 INFO TaskSetManager: Starting task 0.0 in stage 3.0 (TID 8, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:54 INFO Executor: Running task 0.0 in stage 3.0 (TID 8)
      15/06/13 11:06:54 INFO PythonRDD: Times: total = 2, boot = -29, init = 31, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 0.0 in stage 3.0 (TID 8). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 0.0 in stage 3.0 (TID 8) in 10 ms on localhost (1/1)
      15/06/13 11:06:55 INFO TaskSchedulerImpl: Removed TaskSet 3.0, whose tasks have all completed, from pool
      15/06/13 11:06:55 INFO DAGScheduler: ResultStage 3 (runJob at PythonRDD.scala:366) finished in 0.011 s
      15/06/13 11:06:55 INFO DAGScheduler: Job 3 finished: runJob at PythonRDD.scala:366, took 0.035088 s
      15/06/13 11:06:55 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:55 INFO DAGScheduler: Got job 4 (runJob at PythonRDD.scala:366) with 4 output partitions (allowLocal=true)
      15/06/13 11:06:55 INFO DAGScheduler: Final stage: ResultStage 4(runJob at PythonRDD.scala:366)
      15/06/13 11:06:55 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:55 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:55 INFO DAGScheduler: Submitting ResultStage 4 (PythonRDD[5] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:55 INFO MemoryStore: ensureFreeSpace(5120) called with curMem=14460, maxMem=278302556
      15/06/13 11:06:55 INFO MemoryStore: Block broadcast_4 stored as values in memory (estimated size 5.0 KB, free 265.4 MB)
      15/06/13 11:06:55 INFO MemoryStore: ensureFreeSpace(3337) called with curMem=19580, maxMem=278302556
      15/06/13 11:06:55 INFO MemoryStore: Block broadcast_4_piece0 stored as bytes in memory (estimated size 3.3 KB, free 265.4 MB)
      15/06/13 11:06:55 INFO BlockManagerInfo: Added broadcast_4_piece0 in memory on localhost:56872 (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:55 INFO SparkContext: Created broadcast 4 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:55 INFO DAGScheduler: Submitting 4 missing tasks from ResultStage 4 (PythonRDD[5] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:55 INFO TaskSchedulerImpl: Adding task set 4.0 with 4 tasks
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 0.0 in stage 4.0 (TID 9, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 1.0 in stage 4.0 (TID 10, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 2.0 in stage 4.0 (TID 11, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 3.0 in stage 4.0 (TID 12, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO Executor: Running task 3.0 in stage 4.0 (TID 12)
      15/06/13 11:06:55 INFO Executor: Running task 1.0 in stage 4.0 (TID 10)
      15/06/13 11:06:55 INFO Executor: Running task 2.0 in stage 4.0 (TID 11)
      15/06/13 11:06:55 INFO Executor: Running task 0.0 in stage 4.0 (TID 9)
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 1, boot = -79, init = 80, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 3.0 in stage 4.0 (TID 12). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 1, boot = -27, init = 28, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 1.0 in stage 4.0 (TID 10). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 3.0 in stage 4.0 (TID 12) in 11 ms on localhost (1/4)
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 1.0 in stage 4.0 (TID 10) in 14 ms on localhost (2/4)
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 22, boot = 22, init = 0, finish = 0
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 21, boot = 21, init = 0, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 2.0 in stage 4.0 (TID 11). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO Executor: Finished task 0.0 in stage 4.0 (TID 9). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 2.0 in stage 4.0 (TID 11) in 37 ms on localhost (3/4)
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 0.0 in stage 4.0 (TID 9) in 43 ms on localhost (4/4)
      15/06/13 11:06:55 INFO TaskSchedulerImpl: Removed TaskSet 4.0, whose tasks have all completed, from pool
      15/06/13 11:06:55 INFO DAGScheduler: ResultStage 4 (runJob at PythonRDD.scala:366) finished in 0.044 s
      15/06/13 11:06:55 INFO DAGScheduler: Job 4 finished: runJob at PythonRDD.scala:366, took 0.059163 s
      15/06/13 11:06:55 INFO SparkContext: Starting job: runJob at PythonRDD.scala:366
      15/06/13 11:06:55 INFO DAGScheduler: Got job 5 (runJob at PythonRDD.scala:366) with 3 output partitions (allowLocal=true)
      15/06/13 11:06:55 INFO DAGScheduler: Final stage: ResultStage 5(runJob at PythonRDD.scala:366)
      15/06/13 11:06:55 INFO DAGScheduler: Parents of final stage: List()
      15/06/13 11:06:55 INFO DAGScheduler: Missing parents: List()
      15/06/13 11:06:55 INFO DAGScheduler: Submitting ResultStage 5 (PythonRDD[6] at RDD at PythonRDD.scala:43), which has no missing parents
      15/06/13 11:06:55 INFO MemoryStore: ensureFreeSpace(5120) called with curMem=22917, maxMem=278302556
      15/06/13 11:06:55 INFO MemoryStore: Block broadcast_5 stored as values in memory (estimated size 5.0 KB, free 265.4 MB)
      15/06/13 11:06:55 INFO MemoryStore: ensureFreeSpace(3338) called with curMem=28037, maxMem=278302556
      15/06/13 11:06:55 INFO MemoryStore: Block broadcast_5_piece0 stored as bytes in memory (estimated size 3.3 KB, free 265.4 MB)
      15/06/13 11:06:55 INFO BlockManagerInfo: Added broadcast_5_piece0 in memory on localhost:56872 (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:55 INFO SparkContext: Created broadcast 5 from broadcast at DAGScheduler.scala:874
      15/06/13 11:06:55 INFO DAGScheduler: Submitting 3 missing tasks from ResultStage 5 (PythonRDD[6] at RDD at PythonRDD.scala:43)
      15/06/13 11:06:55 INFO TaskSchedulerImpl: Adding task set 5.0 with 3 tasks
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 0.0 in stage 5.0 (TID 13, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 1.0 in stage 5.0 (TID 14, localhost, PROCESS_LOCAL, 1665 bytes)
      15/06/13 11:06:55 INFO TaskSetManager: Starting task 2.0 in stage 5.0 (TID 15, localhost, PROCESS_LOCAL, 1708 bytes)
      15/06/13 11:06:55 INFO Executor: Running task 0.0 in stage 5.0 (TID 13)
      15/06/13 11:06:55 INFO Executor: Running task 1.0 in stage 5.0 (TID 14)
      15/06/13 11:06:55 INFO Executor: Running task 2.0 in stage 5.0 (TID 15)
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 1, boot = -24, init = 25, finish = 0
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 1, boot = -24, init = 25, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 0.0 in stage 5.0 (TID 13). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO Executor: Finished task 2.0 in stage 5.0 (TID 15). 716 bytes result sent to driver
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 0.0 in stage 5.0 (TID 13) in 12 ms on localhost (1/3)
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 2.0 in stage 5.0 (TID 15) in 11 ms on localhost (2/3)
      15/06/13 11:06:55 INFO PythonRDD: Times: total = 21, boot = 20, init = 1, finish = 0
      15/06/13 11:06:55 INFO Executor: Finished task 1.0 in stage 5.0 (TID 14). 666 bytes result sent to driver
      15/06/13 11:06:55 INFO TaskSetManager: Finished task 1.0 in stage 5.0 (TID 14) in 36 ms on localhost (3/3)
      15/06/13 11:06:55 INFO TaskSchedulerImpl: Removed TaskSet 5.0, whose tasks have all completed, from pool
      15/06/13 11:06:55 INFO DAGScheduler: ResultStage 5 (runJob at PythonRDD.scala:366) finished in 0.038 s
      15/06/13 11:06:55 INFO DAGScheduler: Job 5 finished: runJob at PythonRDD.scala:366, took 0.052978 s
      15/06/13 11:06:56 INFO HiveContext: Initializing execution hive, version 0.13.1
      15/06/13 11:06:56 INFO HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
      15/06/13 11:06:56 INFO ObjectStore: ObjectStore, initialize called
      15/06/13 11:06:57 INFO Persistence: Property datanucleus.cache.level2 unknown - will be ignored
      15/06/13 11:06:57 INFO Persistence: Property hive.metastore.integral.jdo.pushdown unknown - will be ignored
      15/06/13 11:06:57 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
      15/06/13 11:06:58 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
      15/06/13 11:06:59 INFO BlockManagerInfo: Removed broadcast_5_piece0 on localhost:56872 in memory (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:59 INFO BlockManagerInfo: Removed broadcast_4_piece0 on localhost:56872 in memory (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:59 INFO BlockManagerInfo: Removed broadcast_3_piece0 on localhost:56872 in memory (size: 3.3 KB, free: 265.4 MB)
      15/06/13 11:06:59 INFO BlockManagerInfo: Removed broadcast_2_piece0 on localhost:56872 in memory (size: 2.3 KB, free: 265.4 MB)
      15/06/13 11:07:00 INFO ObjectStore: Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"
      15/06/13 11:07:00 INFO MetaStoreDirectSql: MySQL check failed, assuming we are not on mysql: Lexical error at line 1, column 5.  Encountered: "@" (64), after : "".
      15/06/13 11:07:01 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:01 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:04 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:04 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:04 INFO ObjectStore: Initialized ObjectStore
      15/06/13 11:07:04 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 0.13.1aa
      15/06/13 11:07:05 INFO HiveMetaStore: Added admin role in metastore
      15/06/13 11:07:05 INFO HiveMetaStore: Added public role in metastore
      15/06/13 11:07:05 INFO HiveMetaStore: No user is added in admin role, since config is empty
      15/06/13 11:07:05 INFO SessionState: No Tez session required at this point. hive.execution.engine=mr.
      15/06/13 11:07:06 INFO HiveContext: Initializing HiveMetastoreConnection version 0.13.1 using Spark classes.
      15/06/13 11:07:07 WARN NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
      15/06/13 11:07:08 INFO HiveMetaStore: 0: Opening raw store with implemenation class:org.apache.hadoop.hive.metastore.ObjectStore
      15/06/13 11:07:08 INFO ObjectStore: ObjectStore, initialize called
      15/06/13 11:07:08 INFO Persistence: Property datanucleus.cache.level2 unknown - will be ignored
      15/06/13 11:07:08 INFO Persistence: Property hive.metastore.integral.jdo.pushdown unknown - will be ignored
      15/06/13 11:07:08 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
      15/06/13 11:07:09 WARN Connection: BoneCP specified but not present in CLASSPATH (or one of dependencies)
      15/06/13 11:07:11 INFO ObjectStore: Setting MetaStore object pin classes with hive.metastore.cache.pinobjtypes="Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"
      15/06/13 11:07:11 INFO MetaStoreDirectSql: MySQL check failed, assuming we are not on mysql: Lexical error at line 1, column 5.  Encountered: "@" (64), after : "".
      15/06/13 11:07:12 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:12 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:13 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MFieldSchema" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:13 INFO Datastore: The class "org.apache.hadoop.hive.metastore.model.MOrder" is tagged as "embedded-only" so does not have its own datastore table.
      15/06/13 11:07:14 INFO ObjectStore: Initialized ObjectStore
      15/06/13 11:07:14 WARN ObjectStore: Version information not found in metastore. hive.metastore.schema.verification is not enabled so recording the schema version 0.13.1aa
      15/06/13 11:07:14 INFO HiveMetaStore: Added admin role in metastore
      15/06/13 11:07:15 INFO HiveMetaStore: Added public role in metastore
      15/06/13 11:07:15 INFO HiveMetaStore: No user is added in admin role, since config is empty
      15/06/13 11:07:15 INFO SessionState: No Tez session required at this point. hive.execution.engine=mr.
      >>>
      >>> df.save("/tmp/d.csv", "com.databricks.spark.csv")
      Traceback (most recent call last):
        File "<stdin>", line 1, in <module>
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/pyspark/sql/dataframe.py", line 202, in save
          return self.write.save(path, source, mode, **options)
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/pyspark/sql/readwriter.py", line 295, in save
          self._jwrite.save(path)
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/lib/py4j-0.8.2.1-src.zip/py4j/java_gateway.py", line 538, in __call__
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py", line 300, in get_return_value
      py4j.protocol.Py4JJavaError: An error occurred while calling o86.save.
      : java.lang.RuntimeException: Failed to load class for data source: com.databricks.spark.csv
      	at scala.sys.package$.error(package.scala:27)
      	at org.apache.spark.sql.sources.ResolvedDataSource$.lookupDataSource(ddl.scala:216)
      	at org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:302)
      	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:144)
      	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:135)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:606)
      	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
      	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
      	at py4j.Gateway.invoke(Gateway.java:259)
      	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
      	at py4j.commands.CallCommand.execute(CallCommand.java:79)
      	at py4j.GatewayConnection.run(GatewayConnection.java:207)
      	at java.lang.Thread.run(Thread.java:744)
      
      >>>
      ...
      >>> df.write.format("com.databricks.spark.csv").save("/tmp/d.csv")
      Traceback (most recent call last):
        File "<stdin>", line 1, in <module>
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/pyspark/sql/readwriter.py", line 295, in save
          self._jwrite.save(path)
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/lib/py4j-0.8.2.1-src.zip/py4j/java_gateway.py", line 538, in __call__
        File "/Users/drake/spark/spark-1.4.0-bin-hadoop2.6/python/lib/py4j-0.8.2.1-src.zip/py4j/protocol.py", line 300, in get_return_value
      py4j.protocol.Py4JJavaError: An error occurred while calling o90.save.
      : java.lang.RuntimeException: Failed to load class for data source: com.databricks.spark.csv
      	at scala.sys.package$.error(package.scala:27)
      	at org.apache.spark.sql.sources.ResolvedDataSource$.lookupDataSource(ddl.scala:216)
      	at org.apache.spark.sql.sources.ResolvedDataSource$.apply(ddl.scala:302)
      	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:144)
      	at org.apache.spark.sql.DataFrameWriter.save(DataFrameWriter.scala:135)
      	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
      	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
      	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
      	at java.lang.reflect.Method.invoke(Method.java:606)
      	at py4j.reflection.MethodInvoker.invoke(MethodInvoker.java:231)
      	at py4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:379)
      	at py4j.Gateway.invoke(Gateway.java:259)
      	at py4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:133)
      	at py4j.commands.CallCommand.execute(CallCommand.java:79)
      	at py4j.GatewayConnection.run(GatewayConnection.java:207)
      	at java.lang.Thread.run(Thread.java:744)
      
      >>>
      

      Attachments

        Issue Links

          Activity

            People

              andrewor14 Andrew Or
              dondrake Don Drake
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: