> On July 30, 2014, 9:08 p.m., Maja Kabiljo wrote: > > Did you test automatic restart of jobs after checkpoint? > > > > Maybe I'm missing something, because it looks to me that after checkpoint > > we finish as if halt was called, meaning we store output. Is this the case?
I tested it with PageRank. After checkpoint we do finish. What's going to happen next depends on how you configure GiraphJobRetryChecker if it is configured to restart the job will restart from the checkpoint. It is not configured this way in open source version (and I'm not sure if it should be) > On July 30, 2014, 9:08 p.m., Maja Kabiljo wrote: > > giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java, lines > > 165-169 > > <https://reviews.apache.org/r/23989/diff/2/?file=643671#file643671line165> > > > > Why do we need two separate flags here? I can't trust flag set by user as it may be set after some workers passed checkpointing stage while others don't. So the logic is: 1) User sets _checkpointAndStop 2) Master checks if _checkpointAndStop is set and sets _forceCheckpointAndStopInternal if so. It happens before barrier 3) After barrier workers check if _forceCheckpointAndStopInternal is set, do checkpoint and stop if thats the case > On July 30, 2014, 9:08 p.m., Maja Kabiljo wrote: > > giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java, line > > 1149 > > <https://reviews.apache.org/r/23989/diff/2/?file=643677#file643677line1149> > > > > What is this? It used to be class name, but not all versions of hadoop support class name. All of them support extensions though > On July 30, 2014, 9:08 p.m., Maja Kabiljo wrote: > > giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java, lines > > 266-281 > > <https://reviews.apache.org/r/23989/diff/2/?file=643680#file643680line266> > > > > Why don't we just use latest checkpoint? Hmm, I guess it's possible, let me check > On July 30, 2014, 9:08 p.m., Maja Kabiljo wrote: > > giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java, > > line 1624 > > <https://reviews.apache.org/r/23989/diff/2/?file=643683#file643683line1624> > > > > Why do we ignore death here? Right, it should be conditioned on checkpointStatus. If it is set to CHECKPOINT_AND_HALT workers will die after checkpointing, we don't wan't master to die in this case. - Sergey ----------------------------------------------------------- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/23989/#review49163 ----------------------------------------------------------- On July 28, 2014, 5:25 p.m., Sergey Edunov wrote: > > ----------------------------------------------------------- > This is an automatically generated e-mail. To reply, visit: > https://reviews.apache.org/r/23989/ > ----------------------------------------------------------- > > (Updated July 28, 2014, 5:25 p.m.) > > > Review request for giraph. > > > Repository: giraph-git > > > Description > ------- > > We need to address some issues with checkpointing: > 1) worker2worker messages are not saved > 2) BspServiceWorker does not compile under hadoop_0.23 profile > 3) it would be nice to be able to manually checkpoint and stop any job at any > point of time. > > Changes: > > 1) worker2worker messages fixed my serializing currentworkertoworker messages > (it is a list of writable so I had to write class information as well) > 2) Compilation issues fixed > 3) The way you can trigger checkpointing now is by creating > /_checkpointAndStop node in zookeeper (same way as _haltComputation works) > After that the behavior of the job will be determined by registered > GiraphJobRetryChecker. By default, job will get checkpointed at the end of > current superstep and halted. You can override this behavior by making > shouldRestartCheckpoint() return true, in this case job will be restarted > immediately after getting checkpointed. > > > Diffs > ----- > > giraph-core/src/main/java/org/apache/giraph/bsp/BspService.java 02577b9 > giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedService.java > ff3e427 > > giraph-core/src/main/java/org/apache/giraph/bsp/CentralizedServiceMaster.java > e5b7cf3 > giraph-core/src/main/java/org/apache/giraph/bsp/CheckpointStatus.java > PRE-CREATION > giraph-core/src/main/java/org/apache/giraph/bsp/SuperstepState.java c384fbf > giraph-core/src/main/java/org/apache/giraph/comm/ServerData.java 29488fc > giraph-core/src/main/java/org/apache/giraph/conf/GiraphConstants.java > 0424a47 > giraph-core/src/main/java/org/apache/giraph/graph/GraphTaskManager.java > 684f4eb > > giraph-core/src/main/java/org/apache/giraph/job/DefaultGiraphJobRetryChecker.java > 0cab86c > giraph-core/src/main/java/org/apache/giraph/job/GiraphJob.java 4a1f02e > giraph-core/src/main/java/org/apache/giraph/job/GiraphJobRetryChecker.java > 53a800e > giraph-core/src/main/java/org/apache/giraph/job/HadoopUtils.java 9530fd6 > giraph-core/src/main/java/org/apache/giraph/master/BspServiceMaster.java > e129390 > giraph-core/src/main/java/org/apache/giraph/master/MasterThread.java > 0635210 > giraph-core/src/main/java/org/apache/giraph/utils/WritableUtils.java > 3f8382e > giraph-core/src/main/java/org/apache/giraph/worker/BspServiceWorker.java > d2d24ee > giraph-core/src/test/java/org/apache/giraph/utils/TestWritableUtils.java > PRE-CREATION > giraph-examples/src/test/java/org/apache/giraph/TestCheckpointing.java > 2939af7 > pom.xml de25499 > > Diff: https://reviews.apache.org/r/23989/diff/ > > > Testing > ------- > > Run pagerank, will keep testing with different jobs. > > > Thanks, > > Sergey Edunov > >
