fix attempts to save dev job status in arvados db
[arvados.git] / services / crunch / crunch-job
index c8c5908ef864fbfa5e747724c783d4ba80386590..8ab0ea377bafca15a5e270b6ec796e54dd8bc43c 100755 (executable)
@@ -10,13 +10,31 @@ crunch-job: Execute job steps, save snapshots as requested, collate output.
 Obtain job details from Arvados, run tasks on compute nodes (typically
 invoked by scheduler on controller):
 
- crunch-job --uuid x-y-z
+ crunch-job --job x-y-z
 
 Obtain job details from command line, run tasks on local machine
 (typically invoked by application or developer on VM):
 
  crunch-job --job '{"script_version":"/path/to/tree","script":"scriptname",...}'
 
+=head1 OPTIONS
+
+=over
+
+=item --force-unlock
+
+If the job is already locked, steal the lock and run it anyway.
+
+=item --git-dir
+
+Path to .git directory where the specified commit is found.
+
+=item --job-api-token
+
+Arvados API authorization token to use during the course of the job.
+
+=back
+
 =head1 RUNNING JOBS LOCALLY
 
 crunch-job's log messages appear on stderr along with the job tasks'
@@ -48,13 +66,13 @@ or unallocated). Currently this is a no-op.
 
 
 use strict;
-use DBI;
 use POSIX ':sys_wait_h';
 use Fcntl qw(F_GETFL F_SETFL O_NONBLOCK);
 use Arvados;
 use Getopt::Long;
 use Warehouse;
 use Warehouse::Stream;
+use IPC::System::Simple qw(capturex);
 
 $ENV{"TMPDIR"} ||= "/tmp";
 $ENV{"CRUNCH_TMP"} = $ENV{"TMPDIR"} . "/crunch-job";
@@ -62,13 +80,21 @@ $ENV{"CRUNCH_WORK"} = $ENV{"CRUNCH_TMP"} . "/work";
 mkdir ($ENV{"CRUNCH_TMP"});
 
 my $force_unlock;
+my $git_dir;
 my $jobspec;
+my $job_api_token;
 my $resume_stash;
 GetOptions('force-unlock' => \$force_unlock,
+           'git-dir=s' => \$git_dir,
            'job=s' => \$jobspec,
+           'job-api-token=s' => \$job_api_token,
            'resume-stash=s' => \$resume_stash,
     );
 
+if (defined $job_api_token) {
+  $ENV{ARVADOS_API_TOKEN} = $job_api_token;
+}
+
 my $have_slurm = exists $ENV{SLURM_JOBID} && exists $ENV{SLURM_NODELIST};
 my $job_has_uuid = $jobspec =~ /^[-a-z\d]+$/;
 
@@ -89,7 +115,7 @@ $SIG{'USR2'} = sub
 
 
 my $arv = Arvados->new;
-my $metastream = Warehouse::Stream->new;
+my $metastream = Warehouse::Stream->new(whc => new Warehouse);
 $metastream->clear;
 $metastream->write_start('log.txt');
 
@@ -128,7 +154,9 @@ else
   }
 
   if (!defined $Job->{'uuid'}) {
-    chomp ($Job->{'uuid'} = sprintf ("%s-t%d-p%d", `hostname -s`, time, $$));
+    my $hostname = `hostname -s`;
+    chomp $hostname;
+    $Job->{'uuid'} = sprintf ("%s-t%d-p%d", $hostname, time, $$);
   }
 }
 $job_id = $Job->{'uuid'};
@@ -222,15 +250,17 @@ if ($job_has_uuid)
   # Claim this job, and make sure nobody else does
 
   $Job->{'is_locked_by'} = $User->{'uuid'};
-  $Job->{'started_at'} = time;
+  $Job->{'started_at'} = gmtime;
   $Job->{'running'} = 1;
   $Job->{'success'} = undef;
   $Job->{'tasks_summary'} = { 'failed' => 0,
                               'todo' => 1,
                               'running' => 0,
                               'done' => 0 };
-  unless ($Job->save() && $Job->{'is_locked_by'} == $User->{'uuid'}) {
-    croak("Error while updating / locking job");
+  if ($job_has_uuid) {
+    unless ($Job->save() && $Job->{'is_locked_by'} == $User->{'uuid'}) {
+      croak("Error while updating / locking job");
+    }
   }
 }
 
@@ -275,9 +305,9 @@ else
     'qsequence' => 0,
     'parameters' => {},
                                                           });
-  push @jobstep, { level => 0,
-                  attempts => 0,
-                   arvados_task => $first_task,
+  push @jobstep, { 'level' => 0,
+                  'attempts' => 0,
+                   'arvados_task' => $first_task,
                 };
   push @jobstep_todo, 0;
 }
@@ -331,9 +361,11 @@ else
   $ENV{"CRUNCH_INSTALL"} = "$ENV{CRUNCH_TMP}/opt";
 
   my $commit;
+  my $git_archive;
   my $treeish = $Job->{'script_version'};
-  my $repo = $ENV{'CRUNCH_DEFAULT_GIT_DIR'};
-  # Todo: let script_version specify alternate repo
+  my $repo = $git_dir || $ENV{'CRUNCH_DEFAULT_GIT_DIR'};
+  # Todo: let script_version specify repository instead of expecting
+  # parent process to figure it out.
   $ENV{"CRUNCH_SRC_URL"} = $repo;
 
   # Create/update our clone of the remote git repo
@@ -380,7 +412,7 @@ else
        Log (undef, "Using commit $commit for tree-ish $treeish");
         if ($commit ne $treeish) {
           $Job->{'script_version'} = $commit;
-          $Job->save() or croak("Error while updating job");
+          !$job_has_uuid or $Job->save() or croak("Error while updating job");
         }
       }
     }
@@ -390,6 +422,7 @@ else
     $ENV{"CRUNCH_SRC_COMMIT"} = $commit;
     @execargs = ("sh", "-c",
                 "mkdir -p $ENV{CRUNCH_INSTALL} && cd $ENV{CRUNCH_TMP} && perl -");
+    $git_archive = `cd $ENV{CRUNCH_SRC} && git archive $commit`;
   }
   else {
     croak ("could not figure out commit id for $treeish");
@@ -398,7 +431,7 @@ else
   my $installpid = fork();
   if ($installpid == 0)
   {
-    srun (\@srunargs, \@execargs, {}, $build_script);
+    srun (\@srunargs, \@execargs, {}, $build_script . $git_archive);
     exit (1);
   }
   while (1)
@@ -414,7 +447,9 @@ else
 
 foreach (qw (script script_version script_parameters resource_limits))
 {
-  Log (undef, $_ . " " . $Job->{$_});
+  Log (undef,
+       "$_ " .
+       (ref($Job->{$_}) ? JSON::encode_json($Job->{$_}) : $Job->{$_}));
 }
 foreach (split (/\n/, $Job->{knobs}))
 {
@@ -627,7 +662,8 @@ for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
     }
 
     # give up if no nodes are succeeding
-    if (!grep { $_->{node}->{losing_streak} == 0 } @slot) {
+    if (!grep { $_->{node}->{losing_streak} == 0 &&
+                    $_->{node}->{hold_count} < 4 } @slot) {
       my $message = "Every node has failed -- giving up on this round";
       Log (undef, $message);
       last THISROUND;
@@ -681,23 +717,25 @@ goto ONELEVEL if !defined $success;
 release_allocation();
 freeze();
 $Job->{'output'} = &collate_output();
-$Job->{'success'} = 0 if !$Job->{'output'};
-$Job->save;
+$Job->{'success'} = $Job->{'output'} && $success;
+$Job->save if $job_has_uuid;
 
 if ($Job->{'output'})
 {
-  $arv->{'collections'}->{'create'}->execute('collection' => {
-    'uuid' => $Job->{'output'},
-    'manifest_text' => system("whget", $Job->{arvados_task}->{output}),
-                                             });;
+  eval {
+    my $manifest_text = capturex("whget", $Job->{'output'});
+    $arv->{'collections'}->{'create'}->execute('collection' => {
+      'uuid' => $Job->{'output'},
+      'manifest_text' => $manifest_text,
+    });
+  };
+  if ($@) {
+    Log (undef, "Failed to register output manifest: $@");
+  }
 }
 
-
 Log (undef, "finish");
 
-$Job->{'success'} = $Job->{'output'} && $success;
-$Job->save;
-
 save_meta();
 exit 0;
 
@@ -714,7 +752,7 @@ sub update_progress_stats
   $Job->{'tasks_summary'}->{'todo'} = $todo;
   $Job->{'tasks_summary'}->{'done'} = $done;
   $Job->{'tasks_summary'}->{'running'} = $running;
-  $Job->save;
+  $Job->save if $job_has_uuid;
   Log (undef, "status: $done done, $running running, $todo todo");
   $progress_is_dirty = 0;
 }
@@ -735,21 +773,23 @@ sub reapchildren
 
   my $exitcode = $?;
   my $exitinfo = "exit $exitcode";
-  $Jobstep->{arvados_task}->reload;
-  my $success = $Jobstep->{arvados_task}->{success};
+  $Jobstep->{'arvados_task'}->reload;
+  my $success = $Jobstep->{'arvados_task'}->{success};
 
   Log ($jobstepid, "child $pid on $whatslot $exitinfo success=$success");
 
   if (!defined $success) {
     # task did not indicate one way or the other --> fail
-    $Jobstep->{arvados_task}->{success} = 0;
-    $Jobstep->{arvados_task}->save;
+    $Jobstep->{'arvados_task'}->{success} = 0;
+    $Jobstep->{'arvados_task'}->save;
     $success = 0;
   }
 
   if (!$success)
   {
-    --$Jobstep->{attempts} if $Jobstep->{node_fail};
+    my $no_incr_attempts;
+    $no_incr_attempts = 1 if $Jobstep->{node_fail};
+
     ++$thisround_failed;
     ++$thisround_failed_multiple if $Jobstep->{attempts} > 1;
 
@@ -762,6 +802,7 @@ sub reapchildren
          $elapsed < 5 &&
          $Jobstep->{attempts} > 1) {
        Log ($jobstepid, "blaming failure on suspect node " . $slot[$proc{$pid}->{slot}]->{node}->{name} . " instead of incrementing jobstep attempts");
+        $no_incr_attempts = 1;
        --$Jobstep->{attempts};
       }
       ban_node_by_slot($proc{$pid}->{slot});
@@ -769,6 +810,8 @@ sub reapchildren
 
     push @jobstep_todo, $jobstepid;
     Log ($jobstepid, "failure in $elapsed seconds");
+
+    --$Jobstep->{attempts} if $no_incr_attempts;
     $Job->{'tasks_summary'}->{'failed'}++;
   }
   else
@@ -782,7 +825,7 @@ sub reapchildren
   $Jobstep->{exitcode} = $exitcode;
   $Jobstep->{finishtime} = time;
   process_stderr ($jobstepid, $success);
-  Log ($jobstepid, "output " . $Jobstep->{arvados_task}->{output});
+  Log ($jobstepid, "output " . $Jobstep->{'arvados_task'}->{output});
 
   close $reader{$jobstepid};
   delete $reader{$jobstepid};
@@ -791,9 +834,12 @@ sub reapchildren
   delete $proc{$pid};
 
   # Load new tasks
-  my $newtask_list = $arv->{'job_tasks'}->{'list'}->execute('where' => {
-    'created_by_job_task' => $Jobstep->{arvados_task}->{uuid}
-                                                             });
+  my $newtask_list = $arv->{'job_tasks'}->{'list'}->execute(
+    'where' => {
+      'created_by_job_task' => $Jobstep->{'arvados_task'}->{uuid}
+    },
+    'order' => 'qsequence'
+  );
   foreach my $arvados_task (@{$newtask_list->{'items'}}) {
     my $jobstep = {
       'level' => $arvados_task->{'sequence'},
@@ -915,12 +961,9 @@ sub preprocess_stderr
 
   while ($jobstep[$job]->{stderr} =~ /^(.*?)\n/) {
     my $line = $1;
-    if ($line =~ /\+\+\+mr/) {
-      last;
-    }
     substr $jobstep[$job]->{stderr}, 0, 1+length($line), "";
     Log ($job, "stderr $line");
-    if ($line =~ /srun: error: (SLURM job $ENV{SLURM_JOBID} has expired) /) {
+    if ($line =~ /srun: error: (SLURM job $ENV{SLURM_JOBID} has expired|Unable to confirm allocation for job) /) {
       # whoa.
       $main::please_freeze = 1;
     }
@@ -952,8 +995,10 @@ sub collate_output
   my $joboutput;
   for (@jobstep)
   {
-    next if !exists $_->{arvados_task}->{output} || $_->{exitcode} != 0;
-    my $output = $_->{arvados_task}->{output};
+    next if (!exists $_->{'arvados_task'}->{output} ||
+             !$_->{'arvados_task'}->{'success'} ||
+             $_->{'exitcode'} != 0);
+    my $output = $_->{'arvados_task'}->{output};
     if ($output !~ /^[0-9a-f]{32}(\+\S+)*$/)
     {
       $output_in_keep ||= $output =~ / [0-9a-f]{32}\S*\+K/;
@@ -981,7 +1026,7 @@ sub collate_output
   {
     Log (undef, "output $joboutput");
     $Job->{'output'} = $joboutput;
-    $Job->save;
+    $Job->save if $job_has_uuid;
   }
   else
   {
@@ -1057,34 +1102,6 @@ sub Log                          # ($jobstep_id, $logmessage)
 }
 
 
-sub reconnect_database
-{
-  return if !$job_has_uuid;
-  return if ($dbh && $dbh->do ("select now()"));
-  for (1..16)
-  {
-    $dbh = DBI->connect(@$Warehouse::Server::DatabaseDSN);
-    if ($dbh) {
-      $dbh->{InactiveDestroy} = 1;
-      return;
-    }
-    warn ($DBI::errstr);
-    sleep $_;
-  }
-  croak ($DBI::errstr) if !$dbh;
-}
-
-
-sub dbh_do
-{
-  return 1 if !$job_has_uuid;
-  my $ret = $dbh->do (@_);
-  return $ret unless (!$ret && $DBI::errstr =~ /server has gone away/);
-  reconnect_database();
-  return $dbh->do (@_);
-}
-
-
 sub croak
 {
   my ($package, $file, $line) = caller;
@@ -1101,11 +1118,11 @@ sub croak
 sub cleanup
 {
   return if !$job_has_uuid;
-  $Job->reload;
+  $Job->reload if $job_has_uuid;
   $Job->{'running'} = 0;
   $Job->{'success'} = 0;
-  $Job->{'finished_at'} = time;
-  $Job->save;
+  $Job->{'finished_at'} = gmtime;
+  $Job->save if $job_has_uuid;
 }
 
 
@@ -1119,7 +1136,7 @@ sub save_meta
   undef $metastream if !$justcheckpoint; # otherwise Log() will try to use it
   Log (undef, "meta key is $loglocator");
   $Job->{'log'} = $loglocator;
-  $Job->save;
+  $Job->save if $job_has_uuid;
 }
 
 
@@ -1156,51 +1173,6 @@ sub freeze
 {
   Log (undef, "Freeze not implemented");
   return;
-
-  my $whc;                      # todo
-  Log (undef, "freeze");
-
-  my $freezer = new Warehouse::Stream (whc => $whc);
-  $freezer->clear;
-  $freezer->name (".");
-  $freezer->write_start ("state.txt");
-
-  $freezer->write_data (join ("\n",
-                             "job $Job->{uuid}",
-                             map
-                             {
-                               $_ . "=" . freezequote($Job->{$_})
-                             } grep { $_ ne "id" } keys %$Job) . "\n\n");
-
-  foreach my $Jobstep (@jobstep)
-  {
-    my $str = join ("\n",
-                   map
-                   {
-                     $_ . "=" . freezequote ($Jobstep->{$_})
-                   } grep {
-                      $_ !~ /^stderr|slotindex|node_fail/
-                    } keys %$Jobstep);
-    $freezer->write_data ($str."\n\n");
-  }
-  if (@jobstep_tomerge)
-  {
-    $freezer->write_data
-       ("merge $jobstep_tomerge_level "
-        . freezequote (join ("\n",
-                             map { freezequote ($_) } @jobstep_tomerge))
-        . "\n\n");
-  }
-
-  $freezer->write_finish;
-  my $frozentokey = $freezer->as_key;
-  undef $freezer;
-  Log (undef, "frozento key is $frozentokey");
-  dbh_do ("update mrjob set frozentokey=? where id=?", undef,
-         $frozentokey, $job_id);
-  my $kfrozentokey = $whc->store_in_keep (hash => $frozentokey, nnodes => 3);
-  Log (undef, "frozento+K key is $kfrozentokey");
-  return $frozentokey;
 }
 
 
@@ -1265,7 +1237,7 @@ sub thaw
   {
     $Job->{$_} = $frozenjob->{$_};
   }
-  $Job->save;
+  $Job->save if $job_has_uuid;
 }
 
 
@@ -1321,6 +1293,7 @@ sub ban_node_by_slot {
   # Don't start any new jobsteps on this node for 60 seconds
   my $slotid = shift;
   $slot[$slotid]->{node}->{hold_until} = 60 + scalar time;
+  $slot[$slotid]->{node}->{hold_count}++;
   Log (undef, "backing off node " . $slot[$slotid]->{node}->{name} . " for 60 seconds");
 }
 
@@ -1341,32 +1314,15 @@ if (readlink ("$destdir.commit") eq $commit) {
     exit 0;
 }
 
+unlink "$destdir.commit";
 open STDOUT, ">", "$destdir.log";
 open STDERR, ">&STDOUT";
 
-if (-d "$destdir/.git") {
-    chdir $destdir or die "chdir $destdir: $!";
-    if (0 != system (qw(git remote set-url origin), $repo)) {
-       # awful... for old versions of git that don't know "remote set-url"
-       shell_or_die (q(perl -pi~ -e '$_="\turl = ).$repo.q(\n" if /url = /' .git/config));
-    }
-}
-elsif ($repo && $commit)
-{
-    shell_or_die('git', 'clone', $repo, $destdir);
-    chdir $destdir or die "chdir $destdir: $!";
-    shell_or_die(qw(git config clean.requireForce false));
-}
-else {
-    die "$destdir does not exist, and no repo/commit specified -- giving up";
-}
-
-if ($commit) {
-    unlink "$destdir.commit";
-    shell_or_die (qw(git stash));
-    shell_or_die (qw(git clean -d -x));
-    shell_or_die (qw(git fetch origin));
-    shell_or_die (qw(git checkout), $commit);
+mkdir $destdir;
+open TARX, "|-", "tar", "-C", $destdir, "-xf", "-";
+print TARX <DATA>;
+if(!close(TARX)) {
+  die "'tar -C $destdir -xf -' exited $?: $!";
 }
 
 my $pwd;
@@ -1398,3 +1354,5 @@ sub shell_or_die
   system (@_) == 0
       or die "@_ failed: $! exit 0x".sprintf("%x",$?);
 }
+
+__DATA__