Merge branch '3198-inode-cache' refs #3198
[arvados.git] / sdk / cli / bin / crunch-job
index c312f5d169fb77b7b853b113ce97195418dd2f56..6ae04812325994678dad9e29d736b5556731f4f4 100755 (executable)
@@ -96,6 +96,7 @@ use File::Temp;
 use Fcntl ':flock';
 use File::Path qw( make_path remove_tree );
 
+use constant TASK_TEMPFAIL => 111;
 use constant EX_TEMPFAIL => 75;
 
 $ENV{"TMPDIR"} ||= "/tmp";
@@ -361,7 +362,7 @@ if (!defined $no_clear_tmp) {
     # TODO: When #5036 is done and widely deployed, we can get rid of the
     # regular expression and just unmount everything with type fuse.keep.
     srun (["srun", "--nodelist=$nodelist", "-D", $ENV{'TMPDIR'}],
-          ['bash', '-ec', 'mount -t fuse,fuse.keep | awk \'($3 ~ /\ykeep\y/){print $3}\' | xargs -r -n 1 fusermount -u -z; sleep 1; rm -rf $JOB_WORK $CRUNCH_INSTALL $CRUNCH_TMP/task $CRUNCH_TMP/src*']);
+          ['bash', '-ec', 'mount -t fuse,fuse.keep | awk \'($3 ~ /\ykeep\y/){print $3}\' | xargs -r -n 1 fusermount -u -z; sleep 1; rm -rf $JOB_WORK $CRUNCH_INSTALL $CRUNCH_TMP/task $CRUNCH_TMP/src* $CRUNCH_TMP/*.cid']);
     exit (1);
   }
   while (1)
@@ -375,7 +376,7 @@ if (!defined $no_clear_tmp) {
 
 # If this job requires a Docker image, install that.
 my $docker_bin = "/usr/bin/docker.io";
-my ($docker_locator, $docker_stream, $docker_hash);
+my ($docker_locator, $docker_stream, $docker_hash, $docker_limitmem);
 if ($docker_locator = $Job->{docker_image_locator}) {
   ($docker_stream, $docker_hash) = find_docker_image($docker_locator);
   if (!$docker_hash)
@@ -384,7 +385,7 @@ if ($docker_locator = $Job->{docker_image_locator}) {
   }
   $docker_stream =~ s/^\.//;
   my $docker_install_script = qq{
-if ! $docker_bin images -q --no-trunc | grep -qxF \Q$docker_hash\E; then
+if ! $docker_bin images -q --no-trunc --all | grep -qxF \Q$docker_hash\E; then
     arv-get \Q$docker_locator$docker_stream/$docker_hash.tar\E | $docker_bin load
 fi
 };
@@ -407,6 +408,12 @@ fi
           .exit_status_s($?));
   }
 
+  # Determine whether this version of Docker supports memory+swap limits.
+  srun(["srun", "--nodelist=" . $node[0]],
+       ["/bin/sh", "-ec", "$docker_bin run --help | grep -qe --memory-swap="],
+      {fork => 1});
+  $docker_limitmem = ($? == 0);
+
   if ($Job->{arvados_sdk_version}) {
     # The job also specifies an Arvados SDK version.  Add the SDKs to the
     # tar file for the build script to install.
@@ -636,12 +643,44 @@ my $thisround_failed_multiple = 0;
 @jobstep_todo = sort { $jobstep[$a]->{level} <=> $jobstep[$b]->{level}
                       or $a <=> $b } @jobstep_todo;
 my $level = $jobstep[$jobstep_todo[0]]->{level};
-Log (undef, "start level $level");
 
+my $initial_tasks_this_level = 0;
+foreach my $id (@jobstep_todo) {
+  $initial_tasks_this_level++ if ($jobstep[$id]->{level} == $level);
+}
+
+# If the number of tasks scheduled at this level #T is smaller than the number
+# of slots available #S, only use the first #T slots, or the first slot on
+# each node, whichever number is greater.
+#
+# When we dispatch tasks later, we'll allocate whole-node resources like RAM
+# based on these numbers.  Using fewer slots makes more resources available
+# to each individual task, which should normally be a better strategy when
+# there are fewer of them running with less parallelism.
+#
+# Note that this calculation is not redone if the initial tasks at
+# this level queue more tasks at the same level.  This may harm
+# overall task throughput for that level.
+my @freeslot;
+if ($initial_tasks_this_level < @node) {
+  @freeslot = (0..$#node);
+} elsif ($initial_tasks_this_level < @slot) {
+  @freeslot = (0..$initial_tasks_this_level - 1);
+} else {
+  @freeslot = (0..$#slot);
+}
+my $round_num_freeslots = scalar(@freeslot);
 
+my %round_max_slots = ();
+for (my $ii = $#freeslot; $ii >= 0; $ii--) {
+  my $this_slot = $slot[$freeslot[$ii]];
+  my $node_name = $this_slot->{node}->{name};
+  $round_max_slots{$node_name} ||= $this_slot->{cpu};
+  last if (scalar(keys(%round_max_slots)) >= @node);
+}
 
+Log(undef, "start level $level with $round_num_freeslots slots");
 my %proc;
-my @freeslot = (0..$#slot);
 my @holdslot;
 my %reader;
 my $progress_is_dirty = 1;
@@ -650,7 +689,6 @@ my $progress_stats_updated = 0;
 update_progress_stats();
 
 
-
 THISROUND:
 for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
 {
@@ -670,6 +708,7 @@ for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
   my $childslotname = join (".",
                            $slot[$childslot]->{node}->{name},
                            $slot[$childslot]->{cpu});
+
   my $childpid = fork();
   if ($childpid == 0)
   {
@@ -703,7 +742,7 @@ for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
     $ENV{"HOME"} = $ENV{"TASK_WORK"};
     $ENV{"TASK_KEEPMOUNT"} = $ENV{"TASK_WORK"}.".keep";
     $ENV{"TASK_TMPDIR"} = $ENV{"TASK_WORK"}; # deprecated
-    $ENV{"CRUNCH_NODE_SLOTS"} = $slot[$childslot]->{node}->{ncpus};
+    $ENV{"CRUNCH_NODE_SLOTS"} = $round_max_slots{$ENV{TASK_SLOT_NODE}};
     $ENV{"PATH"} = $ENV{"CRUNCH_INSTALL"} . "/bin:" . $ENV{"PATH"};
 
     $ENV{"GZIP"} = "-n";
@@ -717,13 +756,26 @@ for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
     my $command =
        "if [ -e $ENV{TASK_WORK} ]; then rm -rf $ENV{TASK_WORK}; fi; "
         ."mkdir -p $ENV{CRUNCH_TMP} $ENV{JOB_WORK} $ENV{TASK_WORK} $ENV{TASK_KEEPMOUNT} "
-       ."&& cd $ENV{CRUNCH_TMP} ";
+       ."&& cd $ENV{CRUNCH_TMP} "
+        # These environment variables get used explicitly later in
+        # $command.  No tool is expected to read these values directly.
+        .q{&& MEM=$(awk '($1 == "MemTotal:"){print $2}' </proc/meminfo) }
+        .q{&& SWAP=$(awk '($1 == "SwapTotal:"){print $2}' </proc/meminfo) }
+        ."&& MEMLIMIT=\$(( (\$MEM * 95) / ($ENV{CRUNCH_NODE_SLOTS} * 100) )) "
+        ."&& let SWAPLIMIT=\$MEMLIMIT+\$SWAP ";
     $command .= "&& exec arv-mount --by-id --allow-other $ENV{TASK_KEEPMOUNT} --exec ";
     if ($docker_hash)
     {
-      my $cidfile = "$ENV{CRUNCH_TMP}/$ENV{TASK_UUID}.cid";
+      my $cidfile = "$ENV{CRUNCH_TMP}/$Jobstep->{arvados_task}->{uuid}-$Jobstep->{failures}.cid";
       $command .= "crunchstat -cgroup-root=/sys/fs/cgroup -cgroup-parent=docker -cgroup-cid=$cidfile -poll=10000 ";
       $command .= "$docker_bin run --rm=true --attach=stdout --attach=stderr --attach=stdin -i --user=crunch --cidfile=$cidfile --sig-proxy ";
+      # We only set memory limits if Docker lets us limit both memory and swap.
+      # Memory limits alone have been supported longer, but subprocesses tend
+      # to get SIGKILL if they exceed that without any swap limit set.
+      # See #5642 for additional background.
+      if ($docker_limitmem) {
+        $command .= "--memory=\${MEMLIMIT}k --memory-swap=\${SWAPLIMIT}k ";
+      }
 
       # Dynamically configure the container to use the host system as its
       # DNS server.  Get the host's global addresses from the ip command,
@@ -829,9 +881,9 @@ for (my $todo_ptr = 0; $todo_ptr <= $#jobstep_todo; $todo_ptr ++)
 
   while (!@freeslot
         ||
-        (@slot > @freeslot && $todo_ptr+1 > $#jobstep_todo))
+        ($round_num_freeslots > @freeslot && $todo_ptr+1 > $#jobstep_todo))
   {
-    last THISROUND if $main::please_freeze;
+    last THISROUND if $main::please_freeze || defined($main::success);
     if ($main::please_info)
     {
       $main::please_info = 0;
@@ -941,7 +993,7 @@ if (!$collated_output) {
   Log (undef, "Failed to write output collection");
 }
 else {
-  Log(undef, "output hash " . $collated_output);
+  Log(undef, "job output $collated_output");
   $Job->update_attributes('output' => $collated_output);
 }
 
@@ -1010,7 +1062,7 @@ sub reapchildren
   {
     my $temporary_fail;
     $temporary_fail ||= $Jobstep->{node_fail};
-    $temporary_fail ||= ($exitvalue == 111);
+    $temporary_fail ||= ($exitvalue == TASK_TEMPFAIL);
 
     ++$thisround_failed;
     ++$thisround_failed_multiple if $Jobstep->{'failures'} >= 1;
@@ -1037,7 +1089,6 @@ sub reapchildren
     if (!$temporary_fail || $Jobstep->{'failures'} >= 3) {
       # Give up on this task, and the whole job
       $main::success = 0;
-      $main::please_freeze = 1;
     }
     # Put this task back on the todo queue
     push @jobstep_todo, $jobstepid;
@@ -1056,7 +1107,9 @@ sub reapchildren
   $Jobstep->{'arvados_task'}->{finished_at} = strftime "%Y-%m-%dT%H:%M:%SZ", gmtime($Jobstep->{finishtime});
   $Jobstep->{'arvados_task'}->save;
   process_stderr ($jobstepid, $task_success);
-  Log ($jobstepid, "output " . $Jobstep->{'arvados_task'}->{output});
+  Log ($jobstepid, sprintf("task output (%d bytes): %s",
+                           length($Jobstep->{'arvados_task'}->{output}),
+                           $Jobstep->{'arvados_task'}->{output}));
 
   close $reader{$jobstepid};
   delete $reader{$jobstepid};
@@ -1230,7 +1283,7 @@ sub preprocess_stderr
       # whoa.
       $main::please_freeze = 1;
     }
-    elsif ($line =~ /srun: error: (Node failure on|Unable to create job step) /) {
+    elsif ($line =~ /(srun: error: (Node failure on|Unable to create job step|.*: Communication connection failure))|arvados.errors.Keep/) {
       $jobstep[$job]->{node_fail} = 1;
       ban_node_by_slot($jobstep[$job]->{slotindex});
     }
@@ -1252,16 +1305,19 @@ sub process_stderr
 sub fetch_block
 {
   my $hash = shift;
-  my ($keep, $child_out, $output_block);
-
-  my $cmd = "arv-get \Q$hash\E";
-  open($keep, '-|', $cmd) or die "fetch_block: $cmd: $!";
-  $output_block = '';
+  my $keep;
+  if (!open($keep, "-|", "arv-get", "--retries", retry_count(), $hash)) {
+    Log(undef, "fetch_block run error from arv-get $hash: $!");
+    return undef;
+  }
+  my $output_block = "";
   while (1) {
     my $buf;
     my $bytes = sysread($keep, $buf, 1024 * 1024);
     if (!defined $bytes) {
-      die "reading from arv-get: $!";
+      Log(undef, "fetch_block read error from arv-get: $!");
+      $output_block = undef;
+      last;
     } elsif ($bytes == 0) {
       # sysread returns 0 at the end of the pipe.
       last;
@@ -1271,6 +1327,10 @@ sub fetch_block
     }
   }
   close $keep;
+  if ($?) {
+    Log(undef, "fetch_block arv-get exited " . exit_status_s($?));
+    $output_block = undef;
+  }
   return $output_block;
 }
 
@@ -1283,50 +1343,64 @@ sub create_output_collection
   Log (undef, "collate");
 
   my ($child_out, $child_in);
-  my $pid = open2($child_out, $child_in, 'python', '-c',
-                  'import arvados; ' .
-                  'import sys; ' .
-                  'print arvados.api()' .
-                  '.collections()' .
-                  '.create(body={"manifest_text":sys.stdin.read()})' .
-                  '.execute()["portable_data_hash"]'
-      );
+  my $pid = open2($child_out, $child_in, 'python', '-c', q{
+import arvados
+import sys
+print (arvados.api("v1").collections().
+       create(body={"manifest_text": sys.stdin.read()}).
+       execute(num_retries=int(sys.argv[1]))["portable_data_hash"])
+}, retry_count());
 
   my $task_idx = -1;
+  my $manifest_size = 0;
   for (@jobstep)
   {
     ++$task_idx;
-    next unless exists $_->{'arvados_task'}->{'output'};
     my $output = $_->{'arvados_task'}->{output};
-    if ($output !~ /^[0-9a-f]{32}(\+\S+)*$/)
-    {
-      print $child_in $output;
-    }
-    elsif (defined (my $outblock = fetch_block ($output)))
-    {
-      print $child_in $outblock;
+    next if (!defined($output));
+    my $next_write;
+    if ($output =~ /^[0-9a-f]{32}(\+\S+)*$/) {
+      $next_write = fetch_block($output);
+    } else {
+      $next_write = $output;
     }
-    else
-    {
+    if (defined($next_write)) {
+      if (!defined(syswrite($child_in, $next_write))) {
+        # There's been an error writing.  Stop the loop.
+        # We'll log details about the exit code later.
+        last;
+      } else {
+        $manifest_size += length($next_write);
+      }
+    } else {
       my $uuid = $_->{'arvados_task'}->{'uuid'};
       Log (undef, "Error retrieving '$output' output by task $task_idx ($uuid)");
       $main::success = 0;
     }
   }
-  $child_in->close;
+  close($child_in);
+  Log(undef, "collated output manifest text to send to API server is $manifest_size bytes with access tokens");
 
   my $joboutput;
   my $s = IO::Select->new($child_out);
   if ($s->can_read(120)) {
-    sysread($child_out, $joboutput, 64 * 1024 * 1024);
-    chomp($joboutput);
-    # TODO: Ensure exit status == 0.
+    sysread($child_out, $joboutput, 1024 * 1024);
+    waitpid($pid, 0);
+    if ($?) {
+      Log(undef, "output collection creation exited " . exit_status_s($?));
+      $joboutput = undef;
+    } else {
+      chomp($joboutput);
+    }
   } else {
     Log (undef, "timed out while creating output collection");
+    foreach my $signal (2, 2, 2, 15, 15, 9) {
+      kill($signal, $pid);
+      last if waitpid($pid, WNOHANG) == -1;
+      sleep(1);
+    }
   }
-  # TODO: kill $pid instead of waiting, now that we've decided to
-  # ignore further output.
-  waitpid($pid, 0);
+  close($child_out);
 
   return $joboutput;
 }
@@ -1400,8 +1474,11 @@ sub log_writer_start($)
 {
   my $logfilename = shift;
   $log_pipe_pid = open2($log_pipe_out, $log_pipe_in,
-                        'arv-put', '--portable-data-hash',
+                        'arv-put',
+                        '--portable-data-hash',
+                        '--project-uuid', $Job->{owner_uuid},
                         '--retries', '3',
+                        '--name', $logfilename,
                         '--filename', $logfilename,
                         '-');
 }
@@ -1805,6 +1882,8 @@ use Fcntl ':flock';
 use File::Path qw( make_path remove_tree );
 use POSIX qw(getcwd);
 
+use constant TASK_TEMPFAIL => 111;
+
 # Map SDK subdirectories to the path environments they belong to.
 my %SDK_ENVVARS = ("perl/lib" => "PERLLIB", "ruby/lib" => "RUBYLIB");
 
@@ -1850,9 +1929,9 @@ if (@ARGV) {
   my $venv_dir = "$job_work/.arvados.venv";
   my $venv_built = -e "$venv_dir/bin/activate";
   if ((!$venv_built) and (-d $python_src) and can_run("virtualenv")) {
-    shell_or_die("virtualenv", "--quiet", "--system-site-packages",
+    shell_or_die(undef, "virtualenv", "--quiet", "--system-site-packages",
                  "--python=python2.7", $venv_dir);
-    shell_or_die("$venv_dir/bin/pip", "--quiet", "install", "-I", $python_src);
+    shell_or_die(TASK_TEMPFAIL, "$venv_dir/bin/pip", "--quiet", "install", "-I", $python_src);
     $venv_built = 1;
     $Log->("Built Python SDK virtualenv");
   }
@@ -1948,12 +2027,12 @@ if ((-d $python_dir) and can_run("python2.7") and
 }
 
 if (-e "$destdir/crunch_scripts/install") {
-    shell_or_die ("$destdir/crunch_scripts/install", $install_dir);
+    shell_or_die (undef, "$destdir/crunch_scripts/install", $install_dir);
 } elsif (!-e "./install.sh" && -e "./tests/autotests.sh") {
     # Old version
-    shell_or_die ("./tests/autotests.sh", $install_dir);
+    shell_or_die (undef, "./tests/autotests.sh", $install_dir);
 } elsif (-e "./install.sh") {
-    shell_or_die ("./install.sh", $install_dir);
+    shell_or_die (undef, "./install.sh", $install_dir);
 }
 
 if ($commit) {
@@ -1974,15 +2053,24 @@ sub can_run {
 
 sub shell_or_die
 {
+  my $exitcode = shift;
+
   if ($ENV{"DEBUG"}) {
     print STDERR "@_\n";
   }
   if (system (@_) != 0) {
     my $err = $!;
-    my $exitstatus = sprintf("exit %d signal %d", $? >> 8, $? & 0x7f);
+    my $code = $?;
+    my $exitstatus = sprintf("exit %d signal %d", $code >> 8, $code & 0x7f);
     open STDERR, ">&STDERR_ORIG";
     system ("cat $destdir.log >&2");
-    die "@_ failed ($err): $exitstatus";
+    warn "@_ failed ($err): $exitstatus";
+    if (defined($exitcode)) {
+      exit $exitcode;
+    }
+    else {
+      exit (($code >> 8) || 1);
+    }
   }
 }