Commit graph

2830 commits

Author SHA1 Message Date
Reynold Xin 56ebfeaa52 Merge pull request #357 from hsaputra/set_boolean_paramname
Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID

Set boolean param name for call to SparkHadoopMapReduceUtil.newTaskAttemptID to make it clear which param being set.
2014-01-08 11:50:06 -08:00
Reynold Xin 5cae05f59e Merge pull request #356 from hsaputra/remove_deprecated_cleanup_method
Remove calls to deprecated mapred's OutputCommitter.cleanupJob

Since Hadoop 1.0.4 the mapred OutputCommitter.commitJob should do cleanup job via call to OutputCommitter.cleanupJob,

Remove SparkHadoopWriter.cleanup since it is used only by PairRDDFunctions.

In fact the implementation of mapred OutputCommitter.commitJob looks like this:

  public void commitJob(JobContext jobContext) throws IOException {
    cleanupJob(jobContext);
  }
2014-01-08 11:47:28 -08:00
walker d942f95d7e Merge remote branch 'upstream/master' 2014-01-09 01:22:26 +08:00
Prashant Sharma 277b4a36c5 we clone hadoop key and values by default and reuse if specified. 2014-01-08 16:32:55 +05:30
Patrick Wendell bc81ce040d Merge remote-tracking branch 'apache-github/master' into standalone-driver
Conflicts:
	core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala
	pom.xml
2014-01-08 00:38:31 -08:00
Henry Saputra aa56585d21 Resolve PR review over 100 chars 2014-01-08 00:38:29 -08:00
Patrick Wendell 3ec21f2eee Show more helpful information in UI 2014-01-08 00:30:10 -08:00
Patrick Wendell c78b381e91 Fixes 2014-01-08 00:09:12 -08:00
Patrick Wendell d0533f7046 Rename to Client 2014-01-07 23:38:51 -08:00
Patrick Wendell 3d939e5fe8 Adding --verbose option to DriverClient 2014-01-07 23:27:18 -08:00
Henry Saputra f6b6f88367 Set boolean param name for two files call to SparkHadoopMapReduceUtil.newTaskAttemptID to make
it clear which param being set.
2014-01-07 23:23:17 -08:00
Henry Saputra 4517326ec6 Remove calls to deprecated mapred's OutputCommitter.cleanupJob because since Hadoop 1.0.4
the mapred OutputCommitter.commitJob should do cleanup job.

In fact the implementation of mapred OutputCommitter.commitJob looks like this:

  public void commitJob(JobContext jobContext) throws IOException {
    cleanupJob(jobContext);
  }

(The jobContext input argument is type of org.apache.hadoop.mapred.JobContext)
2014-01-07 22:55:56 -08:00
Patrick Wendell f5f12dc282 Merge pull request #336 from liancheng/akka-remote-lookup
Get rid of `Either[ActorRef, ActorSelection]'

In this pull request, instead of returning an `Either[ActorRef, ActorSelection]`, `registerOrLookup` identifies the remote actor blockingly to obtain an `ActorRef`, or throws an exception if the remote actor doesn't exist or the lookup times out (configured by `spark.akka.lookupTimeout`).  This function is only called when an `SparkEnv` is constructed (instantiating driver or executor), so the blocking call is considered acceptable.  Executor side `ActorSelection`s/`ActorRef`s to driver side `MapOutputTrackerMasterActor` and `BlockManagerMasterActor` are affected by this pull request.

`ActorSelection` is dangerous and should be used with care.  It's only absolutely safe to send messages via an `ActorSelection` when the remote actor is stateless, so that actor incarnation is irrelevant.  But as pointed by @ScrapCodes in the comments below, executor exits immediately once the connection to the driver lost, `ActorSelection`s are not harmful in this scenario.  So this pull request is mostly a code style patch.
2014-01-07 21:56:35 -08:00
Matei Zaharia d75dc428da Merge pull request #350 from mateiz/standalone-limit
Add way to limit default # of cores used by apps in standalone mode

Also documents the spark.deploy.spreadOut option, and fixes a config option that had a dash in its name.
2014-01-08 00:30:03 -05:00
Patrick Wendell 61674bcadf Merge pull request #352 from markhamstra/oldArch
Don't leave os.arch unset after BlockManagerSuite

Recent SparkConf changes meant that BlockManagerSuite was now leaving the os.arch System.property unset.  That's a problem for any subsequent tests that rely upon having a valid os.arch.  This is true for CompressionCodecSuite in the usual maven build test order, even though it isn't usually true for the sbt build.
2014-01-07 18:32:13 -08:00
Mark Hamstra 86ed1ad252 Fix BlockManagerSuite#after 2014-01-07 16:39:37 -08:00
Matei Zaharia 2c421749ea Address review comments 2014-01-07 19:30:23 -05:00
Patrick Wendell e21a707a13 Adding unit tests and some refactoring to promote testability. 2014-01-07 15:39:47 -08:00
Matei Zaharia 044c8ad3a4 Fix unit test compilation 2014-01-07 16:12:20 -05:00
Patrick Wendell e688e11206 Add log4j exclusion rule to maven.
To make this work I had to rename the defaults file. Otherwise
maven's pattern matching rules included it when trying to match
other log4j.properties files.

I also fixed a bug in the existing maven build where two
<transformers> tags were present in assembly/pom.xml
such that one overwrote the other.
2014-01-07 12:56:24 -08:00
Andrew Or 80ba9f8ba0 Get SparkConf from SparkEnv, rather than creating new ones 2014-01-07 12:44:22 -08:00
Matei Zaharia d8bcc8e9a0 Add way to limit default # of cores used by applications on standalone mode
Also documents the spark.deploy.spreadOut option.
2014-01-07 14:35:52 -05:00
Reynold Xin 15d9534501 Merge pull request #318 from srowen/master
Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance

Sorry if this is too abrupt or not a welcome set of changes, but thought I'd see if I could contribute a little. I'm a Java developer and just getting seriously into Spark. So I thought I'd suggest a number of small changes to the couple Java parts of the code to make it a little tighter, more standard and even a bit faster.

Feel free to take all, some or none of this. Happy to explain any of it.
2014-01-07 08:10:02 -08:00
Prashant Sharma c729fa7c8e formatting related fixes suggested by Patrick. 2014-01-07 13:08:16 +05:30
Prashant Sharma b84dc780d3 Allow configuration to be printed in logs for diagnosis. 2014-01-07 13:01:43 +05:30
Prashant Sharma b3018811e1 Allow users to set arbitrary akka configurations via spark conf. 2014-01-07 13:01:43 +05:30
Patrick Wendell 6a3daead2d Fixes after merge 2014-01-06 20:12:45 -08:00
Patrick Wendell c0498f9265 Merge remote-tracking branch 'apache-github/master' into standalone-driver
Conflicts:
	core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala
	core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala
	core/src/main/scala/org/apache/spark/deploy/master/Master.scala
	core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
2014-01-06 17:29:21 -08:00
Patrick Wendell f236ddd1a2 Changes based on review feedback. 2014-01-06 17:15:52 -08:00
Patrick Wendell 9272a004af Fix test breaking downstream builds 2014-01-06 13:03:19 -08:00
Patrick Wendell 357083c29f Merge pull request #330 from tgravescs/fix_addjars_null_handling
Fix handling of empty SPARK_EXAMPLES_JAR

Currently if SPARK_EXAMPLES_JAR is left unset you get a null pointer exception when running the examples (atleast on spark on yarn).  The null now gets turned into a string of "null" when its put into the SparkConf so addJar no longer properly ignores it. This fixes that so that it can be left unset.
2014-01-06 10:29:04 -08:00
walker 2ad315e80f add inline comments 2014-01-07 01:27:57 +08:00
walker 6ab1db8071 add inline comments 2014-01-07 01:21:25 +08:00
walker a0c6d96e27 Merge remote branch 'upstream/master' 2014-01-07 01:05:18 +08:00
Sean Owen 7379b2915f Merge remote-tracking branch 'upstream/master' 2014-01-06 15:13:16 +00:00
Thomas Graves 25446dd931 Add warning to null setJars check 2014-01-06 07:58:59 -06:00
Tathagata Das ac1f4b06c1 Added a hashmap to cache file mod times. 2014-01-05 23:42:53 -08:00
Patrick Wendell a2e7e04974 Merge pull request #333 from pwendell/logging-silence
Quiet ERROR-level Akka Logs

This fixes an issue I've seen where akka logs a bunch of things at ERROR level when connecting to a standalone cluster, even in the normal case. I noticed that even when lifecycle logging was disabled, the netty code inside of akka still logged away via akka's EndpointWriter class. There are also some other log streams that I think are new in akka 2.2.1 that I've disabled.

Finally, I added some better logging to the standalone client. This makes it more clear when a connection failure occurs what is going on. Previously it never explicitly said if a connection attempt had failed.

The commit messages here have some more detail.
2014-01-05 22:37:36 -08:00
Patrick Wendell 675d7eb4f0 Responding to Aaron's review 2014-01-05 21:23:14 -08:00
Lian, Cheng eb24684748 Fixed test suite compilation errors 2014-01-06 11:26:59 +08:00
Reynold Xin 5b0986a1d6 Merge pull request #334 from pwendell/examples-fix
Removing SPARK_EXAMPLES_JAR in the code

This re-writes all of the examples to use the `SparkContext.jarOfClass` mechanism for loading the examples jar. This necessary for environments like YARN and the Standalone mode where example programs will be submit from inside the cluster rather than at the client using `./spark-example`.

This still leaves SPARK_EXAMPLES_JAR in place in the shell scripts for setting up the classpath if `./spark-example` is run.
2014-01-05 19:25:09 -08:00
Lian, Cheng 5c152e3e21 Fixed several compilation errors in test suites 2014-01-06 10:39:05 +08:00
Tathagata Das 2394794591 Merge branch 'filestream-fix' into driver-test
Conflicts:
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
2014-01-06 02:23:53 +00:00
Tathagata Das 8e88db3ca5 Bug fixes to the DriverRunner and minor changes here and there. 2014-01-06 02:21:56 +00:00
Lian, Cheng a4048ff31e Get rid of `Either[ActorRef, ActorSelection]'
Although we can send messages via an ActorSelection, it would be better to identify the actor and obtain an ActorRef first, so that we can get informed earlier if the remote actor doesn't exist, and get rid of the annoying Either wrapper.
2014-01-06 09:18:17 +08:00
Reynold Xin 63f906322d Fall back to zero-arg constructor for Serializer initialization if there is no constructor that accepts SparkConf.
This maintains backward compatibility with older serializers implemented by users.
2014-01-05 15:52:43 -08:00
Patrick Wendell 94fdcda896 Provide logging when attempts to connect to the master fail.
Without these it's a bit less clear what's going on for the user.

One thing I realize when doing this is that akka itself actually retries
the initial association. So the retry we currently have is redundant with
akka's.
2014-01-05 15:16:01 -08:00
Patrick Wendell aaaa673184 Quite akka when remote lifecycle logging is disabled.
I noticed when connecting to a standalone cluster Spark gives a bunch
of Akka ERROR logs that make it seem like something is failing.

This patch does two things:

1. Akka dead letter logging is turned on/off according to the existing
   lifecycle spark property.
2. We explicitly silence akka's EndpointWriter log in log4j. This is necessary
   because for some reason that log doesn't pick up on the lifecycle
   logging settings. After a few hours of debugging this was the only solution
   I found that worked.
2014-01-05 15:15:59 -08:00
Patrick Wendell 79f52809c8 Removing SPARK_EXAMPLES_JAR in the code 2014-01-05 11:49:42 -08:00
Andrew Or 4de9c9554c Use AtomicInteger for numRunningTasks 2014-01-04 11:16:30 -08:00
Thomas Graves ad35c1a5f2 Fix handling of empty SPARK_EXAMPLES_JAR 2014-01-04 11:42:17 -06:00
Tathagata Das 3d4474330d Removed the exponential backoff for testing. 2014-01-04 08:39:00 -08:00
Andrew Or 2db7884f6f Address Mark's comments 2014-01-04 01:20:09 -08:00
Andrew Or 4296d96c82 Assign spill threshold as a fraction of maximum memory
Further, divide this threshold by the number of tasks running concurrently.

Note that this does not guard against the following scenario: a new task
quickly fills up its share of the memory before old tasks finish spilling
their contents, in which case the total memory used by such maps may exceed
what was specified. Currently, spark.shuffle.safetyFraction mitigates the
effect of this.
2014-01-04 00:00:57 -08:00
Patrick Wendell 604fad9c39 Merge remote-tracking branch 'apache-github/master' into remove-binaries
Conflicts:
	core/src/test/scala/org/apache/spark/DriverSuite.scala
	docs/python-programming-guide.md
2014-01-03 21:29:33 -08:00
Patrick Wendell 9e6f3bdcda Changes on top of Prashant's patch.
Closes #316
2014-01-03 18:30:17 -08:00
Andrew Or 333d58df86 Remove unnecessary ClassTag's 2014-01-03 17:55:26 -08:00
Andrew Or 838b0e7d15 Refactor using SparkConf 2014-01-03 16:13:40 -08:00
Patrick Wendell 4ae101ff38 Merge pull request #317 from ScrapCodes/spark-915-segregate-scripts
Spark-915 segregate scripts
2014-01-03 11:24:35 -08:00
Prashant Sharma 9ae382c363 sbin/compute-classpath* bin/compute-classpath* 2014-01-03 15:12:29 +05:30
Prashant Sharma 74ba97fcf7 sbin/spark-class* -> bin/spark-class* 2014-01-03 15:08:01 +05:30
Prashant Sharma bc311bb826 Restored the previously removed test 2014-01-03 14:52:37 +05:30
Prashant Sharma 94f2fffa23 fixed review comments 2014-01-03 14:43:37 +05:30
Prashant Sharma b4bb80002b Merge branch 'master' into spark-1002-remove-jars 2014-01-03 12:12:04 +05:30
Andrew Or df413e996f Merge remote-tracking branch 'spark/master'
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
2014-01-02 20:51:23 -08:00
Tathagata Das a1b8dd53e3 Added StreamingContext.getOrCreate to for automatic recovery, and added RecoverableNetworkWordCount example to use it. 2014-01-02 19:07:22 -08:00
Reynold Xin 0475ca8f81 Merge pull request #320 from kayousterhout/erroneous_failed_msg
Remove erroneous FAILED state for killed tasks.

Currently, when tasks are killed, the Executor first sends a
status update for the task with a "KILLED" state, and then
sends a second status update with a "FAILED" state saying that
the task failed due to an exception. The second FAILED state is
misleading/unncessary, and occurs due to a NonLocalReturnControl
Exception that gets thrown due to the way we kill tasks. This
commit eliminates that problem.

I'm not at all sure that this is the best way to fix this problem,
so alternate suggestions welcome. @rxin guessing you're the right
person to look at this.
2014-01-02 15:17:08 -08:00
Aaron Davidson 8831923219 TempBlockId takes UUID and is explicitly non-serializable 2014-01-02 13:52:35 -08:00
Patrick Wendell 588a1695f4 Merge pull request #297 from tdas/window-improvement
Improvements to DStream window ops and refactoring of Spark's CheckpointSuite

- Added a new RDD - PartitionerAwareUnionRDD. Using this RDD, one can take multiple RDDs partitioned by the same partitioner and unify them into a single RDD while preserving the partitioner. So m RDDs with p partitions each will be unified to a single RDD with p partitions and the same partitioner. The preferred location for each partition of the unified RDD will be the most common preferred location of the corresponding partitions of the parent RDDs. For example, location of partition 0 of the unified RDD will be where most of partition 0 of the parent RDDs are located.
- Improved the performance of DStream's reduceByKeyAndWindow and groupByKeyAndWindow. Both these operations work by doing per-batch reduceByKey/groupByKey and then using PartitionerAwareUnionRDD to union the RDDs across the window. This eliminates a shuffle related to the window operation, which can reduce batch processing time by 30-40% for simple workloads.
- Fixed bugs and simplified Spark's CheckpointSuite. Some of the tests were incorrect and unreliable. Added missing tests for ZippedRDD. I can go into greater detail if necessary.
- Added mapSideCombine option to combineByKeyAndWindow.
2014-01-02 13:20:54 -08:00
Matei Zaharia 7bafb68d77 Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/incubator-spark 2014-01-02 15:57:28 -05:00
Matei Zaharia ca67909cd4 Merge pull request #311 from tmyklebu/master
SPARK-991: Report information gleaned from a Python stacktrace in the UI

Scala:

- Added setCallSite/clearCallSite to SparkContext and JavaSparkContext.
  These functions mutate a LocalProperty called "externalCallSite."
- Add a wrapper, getCallSite, that checks for an externalCallSite and, if
  none is found, calls the usual Utils.formatSparkCallSite.
- Change everything that calls Utils.formatSparkCallSite to call
  getCallSite instead. Except getCallSite.
- Add wrappers to setCallSite/clearCallSite wrappers to JavaSparkContext.

Python:

- Add a gruesome hack to rdd.py that inspects the traceback and guesses
  what you want to see in the UI.
- Add a RAII wrapper around said gruesome hack that calls
  setCallSite/clearCallSite as appropriate.
- Wire said RAII wrapper up around three calls into the Scala code.
  I'm not sure that I hit all the spots with the RAII wrapper. I'm also
  not sure that my gruesome hack does exactly what we want.

One could also approach this change by refactoring
runJob/submitJob/runApproximateJob to take a call site, then threading
that parameter through everything that needs to know it.

One might object to the pointless-looking wrappers in JavaSparkContext.
Unfortunately, I can't directly access the SparkContext from
Python---or, if I can, I don't know how---so I need to wrap everything
that matters in JavaSparkContext.

Conflicts:
	core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
2014-01-02 15:54:54 -05:00
Kay Ousterhout a1b438d94d Remove erroneous FAILED state for killed tasks.
Currently, when tasks are killed, the Executor first sends a
status update for the task with a "KILLED" state, and then
sends a second status update with a "FAILED" state saying that
the task failed due to an exception. The second FAILED state is
misleading/unncessary, and occurs due to a NonLocalReturnControl
Exception that gets thrown due to the way we kill tasks. This
commit eliminates that problem.
2014-01-02 12:34:46 -08:00
Kay Ousterhout 5a3c00c958 Removed redundant TaskSetManager.error() function.
This function was leftover from a while ago, and now just
passes all calls through to the abort() function, so this
commit deletes it.
2014-01-02 11:13:58 -08:00
Sean Owen 66d501276b Suggested small changes to Java code for slightly more standard style, encapsulation and in some cases performance 2014-01-02 16:17:57 +00:00
Prashant Sharma 980afd280a Merge branch 'scripts-reorg' of github.com:shane-huang/incubator-spark into spark-915-segregate-scripts
Conflicts:
	bin/spark-shell
	core/pom.xml
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
	core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala
	core/src/test/scala/org/apache/spark/DriverSuite.scala
	python/run-tests
	sbin/compute-classpath.sh
	sbin/spark-class
	sbin/stop-slaves.sh
2014-01-02 17:55:21 +05:30
Prashant Sharma 08ec10de17 Removed a repeated test and changed tests to not use uncommons jar 2014-01-02 17:32:11 +05:30
Prashant Sharma 436f3d2856 ignoring tests for now, contrary to what I assumed these tests make sense given what they are testing. 2014-01-02 16:08:35 +05:30
Matei Zaharia 0f6060733d Fixed two uses of conf.get with no default value in Mesos 2014-01-01 22:09:42 -05:00
Matei Zaharia e2c68642c6 Miscellaneous fixes from code review.
Also replaced SparkConf.getOrElse with just a "get" that takes a default
value, and added getInt, getLong, etc to make code that uses this
simpler later on.
2014-01-01 22:03:39 -05:00
Matei Zaharia 45ff8f413d Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	core/src/main/scala/org/apache/spark/SparkContext.scala
	core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala
	core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala
2014-01-01 21:25:00 -05:00
Patrick Wendell f8d245bdfc Merge remote-tracking branch 'apache-github/master' into log4j-fix-2
Conflicts:
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
2014-01-01 16:10:51 -08:00
Andrew Or 92c304fd03 Simplify ExternalAppendOnlyMap on the assumption that the mergeCombiners function is specified 2014-01-01 11:42:33 -08:00
Matei Zaharia 0e5b2adb5c Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	project/SparkBuild.scala
2014-01-01 13:28:54 -05:00
Andrew Or 3bc9e391a3 Merge branch 'master' of github.com:andrewor14/incubator-spark 2013-12-31 20:02:12 -08:00
Andrew Or 83dfa16664 Address Patrick's and Reynold's comments 2013-12-31 20:02:05 -08:00
Reynold Xin 8b8e70ebde Merge pull request #73 from falaki/ApproximateDistinctCount
Approximate distinct count

Added countApproxDistinct() to RDD and countApproxDistinctByKey() to PairRDDFunctions to approximately count distinct number of elements and distinct number of values per key, respectively. Both functions use HyperLogLog from stream-lib for counting. Both functions take a parameter that controls the trade-off between accuracy and memory consumption. Also added Scala docs and test suites for both methods.
2013-12-31 17:48:24 -08:00
Aaron Davidson 08302b113a Rename IntermediateBlockId to TempBlockId 2013-12-31 17:44:15 -08:00
Patrick Wendell 37c43c9dd1 Adding outer checkout when initializing logging 2013-12-31 17:36:56 -08:00
Andrew Or 8bbe08b21e Merge branch 'master' of github.com:andrewor14/incubator-spark 2013-12-31 17:26:26 -08:00
Andrew Or 53d8d36684 Add support and test for null keys in ExternalAppendOnlyMap
Also add safeguard against use of destructively sorted AppendOnlyMap
2013-12-31 17:19:02 -08:00
Hossein Falaki bee445c927 Made the code more compact and readable 2013-12-31 16:58:18 -08:00
Hossein Falaki acb0323053 minor improvements 2013-12-31 15:34:26 -08:00
Matei Zaharia ba9338f104 Merge remote-tracking branch 'apache/master' into conf2
Conflicts:
	core/src/main/scala/org/apache/spark/rdd/CheckpointRDD.scala
	streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala
	streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala
2013-12-31 18:23:14 -05:00
Patrick Wendell 63b411dd86 Merge pull request #238 from ngbinh/upgradeNetty
upgrade Netty from 4.0.0.Beta2 to 4.0.13.Final

the changes are listed at https://github.com/netty/netty/wiki/New-and-noteworthy
2013-12-31 14:31:28 -08:00
Andrew Or 3ce22df954 Add warning message for spilling 2013-12-31 11:33:10 -08:00
Andrew Or 94ddc91d06 Address Aaron's and Jerry's comments 2013-12-31 10:50:08 -08:00
Patrick Wendell 55b7e2fdff Merge pull request #289 from tdas/filestream-fix
Bug fixes for file input stream and checkpointing

- Fixed bugs in the file input stream that led the stream to fail due to transient HDFS errors (listing files when a background thread it deleting fails caused errors, etc.)
- Updated Spark's CheckpointRDD and Streaming's CheckpointWriter to use SparkContext.hadoopConfiguration, to allow checkpoints to be written to any HDFS compatible store requiring special configuration.
- Changed the API of SparkContext.setCheckpointDir() - eliminated the unnecessary 'useExisting' parameter. Now SparkContext will always create a unique subdirectory within the user specified checkpoint directory. This is to ensure that previous checkpoint files are not accidentally overwritten.
- Fixed bug where setting checkpoint directory as a relative local path caused the checkpointing to fail.
2013-12-31 10:12:51 -08:00
Tathagata Das fcd17a1e8e Fixed comments and long lines based on comments on PR 289. 2013-12-31 02:01:45 -08:00
Patrick Wendell 4abb0c57ab Tiny typo fix 2013-12-31 00:05:03 -08:00
Patrick Wendell 4d009dcac6 Removing use in test 2013-12-31 00:01:44 -08:00