All of lore.kernel.org
 help / color / mirror / Atom feed
* [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update"
@ 2015-09-23  1:45 Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 01/14] submodule: Send "Fetching submodule <foo>" to standard error Stefan Beller
                   ` (13 more replies)
  0 siblings, 14 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Today there was lots of discussion on the correct way of reading the
strbufs as well as some discussion on the structure of the
asynchronous parallel process loop.

Patches 1-8 bring parallel fetching of submodules, and have had some good exposure
to review and feedback is incorporated.

Patches 9-14 bring parallel submodule updates.
Patch 14 is not ready yet (i.e. test suite failures), but the cleanups before
in patch 9-13 can be reviewed without wasting time.

Any feedback welcome,
Thanks,
Stefan

Diff to v3 below. The patches can also be found at [1]
[1] https://github.com/stefanbeller/git/tree/submodulec_nonthreaded_parallel_4

Jonathan Nieder (1):
  submodule: Send "Fetching submodule <foo>" to standard error

Stefan Beller (13):
  xread: poll on non blocking fds
  xread_nonblock: add functionality to read from fds without blocking
  strbuf: add strbuf_read_once to read without blocking
  run-command: factor out return value computation
  run-command: add an asynchronous parallel child processor
  fetch_populated_submodules: use new parallel job processing
  submodules: allow parallel fetching, add tests and documentation
  submodule-config: Untangle logic in parse_config
  submodule config: keep update strategy around
  git submodule update: cmd_update_recursive
  git submodule update: cmd_update_clone
  git submodule update: cmd_update_fetch
  Rewrite submodule update in C

 Documentation/fetch-options.txt |   7 +
 builtin/fetch.c                 |   6 +-
 builtin/pull.c                  |   6 +
 builtin/submodule--helper.c     | 251 +++++++++++++++++++++++++++++
 git-compat-util.h               |   1 +
 git-submodule.sh                | 339 ++++++++++++++--------------------------
 run-command.c                   | 320 ++++++++++++++++++++++++++++++++++---
 run-command.h                   |  36 +++++
 strbuf.c                        |  11 ++
 strbuf.h                        |   9 ++
 submodule-config.c              |  85 +++++-----
 submodule-config.h              |   1 +
 submodule.c                     | 120 +++++++++-----
 submodule.h                     |   2 +-
 t/t0061-run-command.sh          |  20 +++
 t/t5526-fetch-submodules.sh     |  70 ++++++---
 test-run-command.c              |  24 +++
 wrapper.c                       |  35 ++++-
 18 files changed, 987 insertions(+), 356 deletions(-)

diff --git a/builtin/submodule--helper.c b/builtin/submodule--helper.c
index baa7563..b79117a 100644
--- a/builtin/submodule--helper.c
+++ b/builtin/submodule--helper.c
@@ -382,10 +382,14 @@ static int update_next_task(void *data,
 		argv_array_pushf(&cp->env_array, "sm_path=%s", sub->path);
 		argv_array_pushf(&cp->env_array, "name=%s", sub->name);
 		argv_array_pushf(&cp->env_array, "url=%s", sub->url);
+		argv_array_pushf(&cp->env_array, "sha1=%s", sha1_to_hex(ce->sha1));
 		argv_array_pushf(&cp->env_array, "update_module=%s", update_module);
 
 		cp->git_cmd = 1;
+		cp->no_stdin = 1;
 		cp->stdout_to_stderr = 1;
+		cp->err = -1;
+		argv_array_init(&cp->args);
 		argv_array_push(&cp->args, "submodule");
 		if (!file_exists(sm_gitdir))
 			argv_array_push(&cp->args, "update_clone");
diff --git a/run-command.c b/run-command.c
index 06d5a5d..494e1f8 100644
--- a/run-command.c
+++ b/run-command.c
@@ -276,8 +276,10 @@ static int wait_or_whine(pid_t pid, const char *argv0)
 		failed_errno = errno;
 		error("waitpid for %s failed: %s", argv0, strerror(errno));
 	} else {
-		if (waiting != pid
-		   || (determine_return_value(status, &code, &failed_errno, argv0) < 0))
+		if (waiting != pid || (determine_return_value(status,
+							      &code,
+							      &failed_errno,
+							      argv0) < 0))
 			error("waitpid is confused (%s)", argv0);
 	}
 
@@ -870,7 +872,6 @@ struct parallel_processes {
 
 	int max_processes;
 	int nr_processes;
-	unsigned all_tasks_started : 1;
 
 	get_next_task_fn get_next_task;
 	start_failure_fn start_failure;
@@ -899,9 +900,9 @@ void default_start_failure(void *data,
 	struct strbuf sb = STRBUF_INIT;
 
 	for (i = 0; cp->argv[i]; i++)
-		strbuf_addf(&sb, "%s ", cp->argv[i]);
+		strbuf_addf(&sb, " %s", cp->argv[i]);
 
-	die_errno("Starting a child failed:\n%s", sb.buf);
+	die_errno("Starting a child failed:%s", sb.buf);
 }
 
 void default_return_value(void *data,
@@ -915,12 +916,12 @@ void default_return_value(void *data,
 		return;
 
 	for (i = 0; cp->argv[i]; i++)
-		strbuf_addf(&sb, "%s ", cp->argv[i]);
+		strbuf_addf(&sb, " %s", cp->argv[i]);
 
-	die_errno("A child failed with return code:\n%s\n%d", sb.buf, result);
+	die_errno("A child failed with return code %d:%s", result, sb.buf);
 }
 
-static void run_processes_parallel_init(struct parallel_processes *pp,
+static void pp_init(struct parallel_processes *pp,
 					int n, void *data,
 					get_next_task_fn get_next_task,
 					start_failure_fn start_failure,
@@ -941,7 +942,6 @@ static void run_processes_parallel_init(struct parallel_processes *pp,
 	pp->return_value = return_value ? return_value : default_return_value;
 
 	pp->nr_processes = 0;
-	pp->all_tasks_started = 0;
 	pp->output_owner = 0;
 	pp->children = xcalloc(n, sizeof(*pp->children));
 	pp->pfd = xcalloc(n, sizeof(*pp->pfd));
@@ -954,9 +954,10 @@ static void run_processes_parallel_init(struct parallel_processes *pp,
 	}
 }
 
-static void run_processes_parallel_cleanup(struct parallel_processes *pp)
+static void pp_cleanup(struct parallel_processes *pp)
 {
 	int i;
+
 	for (i = 0; i < pp->max_processes; i++)
 		strbuf_release(&pp->children[i].err);
 
@@ -976,7 +977,8 @@ static void set_nonblocking(int fd)
 			"output will be degraded");
 }
 
-static void run_processes_parallel_start_one(struct parallel_processes *pp)
+/* returns 1 if a process was started, 0 otherwise */
+static int pp_start_one(struct parallel_processes *pp)
 {
 	int i;
 
@@ -988,10 +990,9 @@ static void run_processes_parallel_start_one(struct parallel_processes *pp)
 
 	if (!pp->get_next_task(pp->data,
 			       &pp->children[i].process,
-			       &pp->children[i].err)) {
-		pp->all_tasks_started = 1;
-		return;
-	}
+			       &pp->children[i].err))
+		return 1;
+
 	if (start_command(&pp->children[i].process))
 		pp->start_failure(pp->data,
 				  &pp->children[i].process,
@@ -1002,23 +1003,17 @@ static void run_processes_parallel_start_one(struct parallel_processes *pp)
 	pp->nr_processes++;
 	pp->children[i].in_use = 1;
 	pp->pfd[i].fd = pp->children[i].process.err;
+	return 0;
 }
 
-static void run_processes_parallel_start_as_needed(struct parallel_processes *pp)
-{
-	while (pp->nr_processes < pp->max_processes &&
-	       !pp->all_tasks_started)
-		run_processes_parallel_start_one(pp);
-}
-
-static void run_processes_parallel_buffer_stderr(struct parallel_processes *pp)
+static void pp_buffer_stderr(struct parallel_processes *pp)
 {
 	int i;
 
 	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
 		if (errno == EINTR)
 			continue;
-		run_processes_parallel_cleanup(pp);
+		pp_cleanup(pp);
 		die_errno("poll");
 	}
 
@@ -1033,7 +1028,7 @@ static void run_processes_parallel_buffer_stderr(struct parallel_processes *pp)
 	}
 }
 
-static void run_processes_parallel_output(struct parallel_processes *pp)
+static void pp_output(struct parallel_processes *pp)
 {
 	int i = pp->output_owner;
 	if (pp->children[i].in_use &&
@@ -1043,7 +1038,7 @@ static void run_processes_parallel_output(struct parallel_processes *pp)
 	}
 }
 
-static void run_processes_parallel_collect_finished(struct parallel_processes *pp)
+static void pp_collect_finished(struct parallel_processes *pp)
 {
 	int i = 0;
 	pid_t pid;
@@ -1063,17 +1058,11 @@ static void run_processes_parallel_collect_finished(struct parallel_processes *p
 			    pid == pp->children[i].process.pid)
 				break;
 		if (i == pp->max_processes)
-			/*
-			 * waitpid returned another process id
-			 * which we are not waiting for.
-			 */
-			return;
-
-		if (strbuf_read_once(&pp->children[i].err,
-				     pp->children[i].process.err, 0) < 0 &&
-		    errno != EAGAIN)
-			die_errno("strbuf_read_once");
+			die("BUG: found a child process we were not aware of");
 
+		if (strbuf_read(&pp->children[i].err,
+				pp->children[i].process.err, 0) < 0)
+			die_errno("strbuf_read");
 
 		if (determine_return_value(wait_status, &code, &errno,
 					   pp->children[i].process.argv[0]) < 0)
@@ -1122,18 +1111,20 @@ int run_processes_parallel(int n, void *data,
 			   return_value_fn return_value)
 {
 	struct parallel_processes pp;
-	run_processes_parallel_init(&pp, n, data,
-				    get_next_task,
-				    start_failure,
-				    return_value);
-
-	while (!pp.all_tasks_started || pp.nr_processes > 0) {
-		run_processes_parallel_start_as_needed(&pp);
-		run_processes_parallel_buffer_stderr(&pp);
-		run_processes_parallel_output(&pp);
-		run_processes_parallel_collect_finished(&pp);
+	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
+
+	while (1) {
+		while (pp.nr_processes < pp.max_processes &&
+		       !pp_start_one(&pp))
+			; /* nothing */
+		if (!pp.nr_processes)
+			break;
+		pp_buffer_stderr(&pp);
+		pp_output(&pp);
+		pp_collect_finished(&pp);
 	}
-	run_processes_parallel_cleanup(&pp);
+
+	pp_cleanup(&pp);
 
 	return 0;
 }
diff --git a/run-command.h b/run-command.h
index 0c1b363..3807fd1 100644
--- a/run-command.h
+++ b/run-command.h
@@ -155,6 +155,4 @@ int run_processes_parallel(int n, void *data,
 			   start_failure_fn,
 			   return_value_fn);
 
-void run_processes_parallel_schedule_error(struct strbuf *err);
-
 #endif
diff --git a/strbuf.h b/strbuf.h
index 4d4e5b1..ea69665 100644
--- a/strbuf.h
+++ b/strbuf.h
@@ -367,8 +367,11 @@ extern size_t strbuf_fread(struct strbuf *, size_t, FILE *);
 extern ssize_t strbuf_read(struct strbuf *, int fd, size_t hint);
 
 /**
- * Same as strbuf_read, just returns non-blockingly by ignoring EAGAIN.
- * The fd must have set O_NONBLOCK.
+ * Read from a file descriptor that is marked as O_NONBLOCK without
+ * blocking.  Returns the number of new bytes appended to the sb.
+ * Negative return value signals there was an error returned from
+ * underlying read(2), in which case the caller should check errno.
+ * e.g. errno == EAGAIN when the read may have blocked.
  */
 extern ssize_t strbuf_read_once(struct strbuf *, int fd, size_t hint);
 
diff --git a/submodule-config.c b/submodule-config.c
index 0298a60..8b8c7d1 100644
--- a/submodule-config.c
+++ b/submodule-config.c
@@ -258,93 +258,72 @@ static int parse_config(const char *var, const char *value, void *data)
 	if (!name_and_item_from_var(var, &name, &item))
 		return 0;
 
-	submodule = lookup_or_create_by_name(me->cache, me->gitmodules_sha1,
-			name.buf);
+	submodule = lookup_or_create_by_name(me->cache,
+					     me->gitmodules_sha1,
+					     name.buf);
 
 	if (!strcmp(item.buf, "path")) {
-		struct strbuf path = STRBUF_INIT;
-		if (!value) {
+		if (!value)
 			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (!me->overwrite && submodule->path != NULL) {
+		else if (!me->overwrite && submodule->path != NULL)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"path");
-			goto release_return;
+		else {
+			if (submodule->path)
+				cache_remove_path(me->cache, submodule);
+			free((void *) submodule->path);
+			submodule->path = xstrdup(value);
+			cache_put_path(me->cache, submodule);
 		}
-
-		if (submodule->path)
-			cache_remove_path(me->cache, submodule);
-		free((void *) submodule->path);
-		strbuf_addstr(&path, value);
-		submodule->path = strbuf_detach(&path, NULL);
-		cache_put_path(me->cache, submodule);
 	} else if (!strcmp(item.buf, "fetchrecursesubmodules")) {
 		/* when parsing worktree configurations we can die early */
 		int die_on_error = is_null_sha1(me->gitmodules_sha1);
 		if (!me->overwrite &&
-		    submodule->fetch_recurse != RECURSE_SUBMODULES_NONE) {
+		    submodule->fetch_recurse != RECURSE_SUBMODULES_NONE)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"fetchrecursesubmodules");
-			goto release_return;
-		}
-
-		submodule->fetch_recurse = parse_fetch_recurse(var, value,
+		else
+			submodule->fetch_recurse = parse_fetch_recurse(
+								var, value,
 								die_on_error);
 	} else if (!strcmp(item.buf, "ignore")) {
-		struct strbuf ignore = STRBUF_INIT;
-		if (!me->overwrite && submodule->ignore != NULL) {
+		if (!value)
+			ret = config_error_nonbool(var);
+		else if (!me->overwrite && submodule->ignore != NULL)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"ignore");
-			goto release_return;
-		}
-		if (!value) {
-			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (strcmp(value, "untracked") && strcmp(value, "dirty") &&
-		    strcmp(value, "all") && strcmp(value, "none")) {
+		else if (strcmp(value, "untracked") &&
+			 strcmp(value, "dirty") &&
+			 strcmp(value, "all") &&
+			 strcmp(value, "none"))
 			warning("Invalid parameter '%s' for config option "
 					"'submodule.%s.ignore'", value, var);
-			goto release_return;
+		else {
+			free((void *) submodule->ignore);
+			submodule->ignore = xstrdup(value);
 		}
-
-		free((void *) submodule->ignore);
-		strbuf_addstr(&ignore, value);
-		submodule->ignore = strbuf_detach(&ignore, NULL);
 	} else if (!strcmp(item.buf, "url")) {
-		struct strbuf url = STRBUF_INIT;
 		if (!value) {
 			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (!me->overwrite && submodule->url != NULL) {
+		} else if (!me->overwrite && submodule->url != NULL) {
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"url");
-			goto release_return;
+		} else {
+			free((void *) submodule->url);
+			submodule->url = xstrdup(value);
 		}
-
-		free((void *) submodule->url);
-		strbuf_addstr(&url, value);
-		submodule->url = strbuf_detach(&url, NULL);
 	} else if (!strcmp(item.buf, "update")) {
-		struct strbuf update = STRBUF_INIT;
-		if (!value) {
+		if (!value)
 			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (!me->overwrite && submodule->update != NULL) {
+		else if (!me->overwrite && submodule->update != NULL)
 			warn_multiple_config(me->commit_sha1, submodule->name,
-					"update");
-			goto release_return;
+					     "update");
+		else {
+			free((void *)submodule->update);
+			submodule->update = xstrdup(value);
 		}
-
-		free((void *) submodule->update);
-		strbuf_addstr(&update, value);
-		submodule->update = strbuf_detach(&update, NULL);
 	}
 
-release_return:
 	strbuf_release(&name);
 	strbuf_release(&item);
 
diff --git a/submodule.c b/submodule.c
index d15364f..fdaf3e4 100644
--- a/submodule.c
+++ b/submodule.c
@@ -650,10 +650,12 @@ int fetch_populated_submodules(const struct argv_array *options,
 {
 	int i;
 	struct submodule_parallel_fetch spf = SPF_INIT;
+
 	spf.work_tree = get_git_work_tree();
 	spf.command_line_option = command_line_option;
 	spf.quiet = quiet;
 	spf.prefix = prefix;
+
 	if (!spf.work_tree)
 		goto out;
 
@@ -738,12 +740,11 @@ int get_next_submodule(void *data, struct child_process *cp,
 		if (is_directory(git_dir)) {
 			child_process_init(cp);
 			cp->dir = strbuf_detach(&submodule_path, NULL);
+			cp->env = local_repo_env;
 			cp->git_cmd = 1;
-			cp->no_stdout = 1;
 			cp->no_stdin = 1;
 			cp->stdout_to_stderr = 1;
 			cp->err = -1;
-			cp->env = local_repo_env;
 			if (!spf->quiet)
 				strbuf_addf(err, "Fetching submodule %s%s\n",
 					    spf->prefix, ce->name);
diff --git a/wrapper.c b/wrapper.c
index 54ce231..41a21e1 100644
--- a/wrapper.c
+++ b/wrapper.c
@@ -206,16 +206,10 @@ ssize_t xread(int fd, void *buf, size_t len)
 				continue;
 			if (errno == EAGAIN || errno == EWOULDBLOCK) {
 				struct pollfd pfd;
-				int i;
 				pfd.events = POLLIN;
 				pfd.fd = fd;
-				i = poll(&pfd, 1, 100);
-				if (i < 0) {
-					if (errno == EINTR || errno == ENOMEM)
-						continue;
-					else
-						die_errno("poll");
-				}
+				/* We deliberately ignore the return value */
+				poll(&pfd, 1, -1);
 			}
 		}
 		return nr;
@@ -225,13 +219,13 @@ ssize_t xread(int fd, void *buf, size_t len)
 /*
  * xread_nonblock() is the same a read(), but it automatically restarts read()
  * interrupted operations (EINTR). xread_nonblock() DOES NOT GUARANTEE that
- * "len" bytes is read even if the data is available.
+ * "len" bytes is read. EWOULDBLOCK is turned into EAGAIN.
  */
 ssize_t xread_nonblock(int fd, void *buf, size_t len)
 {
 	ssize_t nr;
 	if (len > MAX_IO_SIZE)
-	    len = MAX_IO_SIZE;
+		len = MAX_IO_SIZE;
 	while (1) {
 		nr = read(fd, buf, len);
 		if (nr < 0) {

-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 01/14] submodule: Send "Fetching submodule <foo>" to standard error
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 02/14] xread: poll on non blocking fds Stefan Beller
                   ` (12 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

From: Jonathan Nieder <jrnieder@gmail.com>

The "Pushing submodule <foo>" progress output correctly goes to
stderr, but "Fetching submodule <foo>" is going to stdout by mistake.
Fix it to write to stderr.

Noticed while trying to implement a parallel submodule fetch.  When
this particular output line went to a different file descriptor, it
was buffered separately, resulting in wrongly interleaved output if
we copied it to the terminal naively.

Signed-off-by: Jonathan Nieder <jrnieder@gmail.com>
Signed-off-by: Stefan Beller <sbeller@google.com>
---
 submodule.c                 |  2 +-
 t/t5526-fetch-submodules.sh | 51 +++++++++++++++++++++++----------------------
 2 files changed, 27 insertions(+), 26 deletions(-)

diff --git a/submodule.c b/submodule.c
index 9fcc86f..1d64e57 100644
--- a/submodule.c
+++ b/submodule.c
@@ -694,7 +694,7 @@ int fetch_populated_submodules(const struct argv_array *options,
 			git_dir = submodule_git_dir.buf;
 		if (is_directory(git_dir)) {
 			if (!quiet)
-				printf("Fetching submodule %s%s\n", prefix, ce->name);
+				fprintf(stderr, "Fetching submodule %s%s\n", prefix, ce->name);
 			cp.dir = submodule_path.buf;
 			argv_array_push(&argv, default_argv);
 			argv_array_push(&argv, "--submodule-prefix");
diff --git a/t/t5526-fetch-submodules.sh b/t/t5526-fetch-submodules.sh
index a4532b0..17759b1 100755
--- a/t/t5526-fetch-submodules.sh
+++ b/t/t5526-fetch-submodules.sh
@@ -16,7 +16,8 @@ add_upstream_commit() {
 		git add subfile &&
 		git commit -m new subfile &&
 		head2=$(git rev-parse --short HEAD) &&
-		echo "From $pwd/submodule" > ../expect.err &&
+		echo "Fetching submodule submodule" > ../expect.err &&
+		echo "From $pwd/submodule" >> ../expect.err &&
 		echo "   $head1..$head2  master     -> origin/master" >> ../expect.err
 	) &&
 	(
@@ -27,6 +28,7 @@ add_upstream_commit() {
 		git add deepsubfile &&
 		git commit -m new deepsubfile &&
 		head2=$(git rev-parse --short HEAD) &&
+		echo "Fetching submodule submodule/subdir/deepsubmodule" >> ../expect.err
 		echo "From $pwd/deepsubmodule" >> ../expect.err &&
 		echo "   $head1..$head2  master     -> origin/master" >> ../expect.err
 	)
@@ -56,9 +58,7 @@ test_expect_success setup '
 	(
 		cd downstream &&
 		git submodule update --init --recursive
-	) &&
-	echo "Fetching submodule submodule" > expect.out &&
-	echo "Fetching submodule submodule/subdir/deepsubmodule" >> expect.out
+	)
 '
 
 test_expect_success "fetch --recurse-submodules recurses into submodules" '
@@ -67,7 +67,7 @@ test_expect_success "fetch --recurse-submodules recurses into submodules" '
 		cd downstream &&
 		git fetch --recurse-submodules >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -96,7 +96,7 @@ test_expect_success "using fetchRecurseSubmodules=true in .gitmodules recurses i
 		git config -f .gitmodules submodule.submodule.fetchRecurseSubmodules true &&
 		git fetch >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -127,7 +127,7 @@ test_expect_success "--recurse-submodules overrides fetchRecurseSubmodules setti
 		git config --unset -f .gitmodules submodule.submodule.fetchRecurseSubmodules &&
 		git config --unset submodule.submodule.fetchRecurseSubmodules
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -146,7 +146,7 @@ test_expect_success "--dry-run propagates to submodules" '
 		cd downstream &&
 		git fetch --recurse-submodules --dry-run >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -155,7 +155,7 @@ test_expect_success "Without --dry-run propagates to submodules" '
 		cd downstream &&
 		git fetch --recurse-submodules >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -166,7 +166,7 @@ test_expect_success "recurseSubmodules=true propagates into submodules" '
 		git config fetch.recurseSubmodules true
 		git fetch >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -180,7 +180,7 @@ test_expect_success "--recurse-submodules overrides config in submodule" '
 		) &&
 		git fetch --recurse-submodules >../actual.out 2>../actual.err
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -214,16 +214,15 @@ test_expect_success "Recursion stops when no new submodule commits are fetched"
 	git add submodule &&
 	git commit -m "new submodule" &&
 	head2=$(git rev-parse --short HEAD) &&
-	echo "Fetching submodule submodule" > expect.out.sub &&
 	echo "From $pwd/." > expect.err.sub &&
 	echo "   $head1..$head2  master     -> origin/master" >>expect.err.sub &&
-	head -2 expect.err >> expect.err.sub &&
+	head -3 expect.err >> expect.err.sub &&
 	(
 		cd downstream &&
 		git fetch >../actual.out 2>../actual.err
 	) &&
 	test_i18ncmp expect.err.sub actual.err &&
-	test_i18ncmp expect.out.sub actual.out
+	test_must_be_empty actual.out
 '
 
 test_expect_success "Recursion doesn't happen when new superproject commits don't change any submodules" '
@@ -269,7 +268,7 @@ test_expect_success "Recursion picks up config in submodule" '
 		)
 	) &&
 	test_i18ncmp expect.err.sub actual.err &&
-	test_i18ncmp expect.out actual.out
+	test_must_be_empty actual.out
 '
 
 test_expect_success "Recursion picks up all submodules when necessary" '
@@ -285,7 +284,8 @@ test_expect_success "Recursion picks up all submodules when necessary" '
 		git add subdir/deepsubmodule &&
 		git commit -m "new deepsubmodule"
 		head2=$(git rev-parse --short HEAD) &&
-		echo "From $pwd/submodule" > ../expect.err.sub &&
+		echo "Fetching submodule submodule" > ../expect.err.sub &&
+		echo "From $pwd/submodule" >> ../expect.err.sub &&
 		echo "   $head1..$head2  master     -> origin/master" >> ../expect.err.sub
 	) &&
 	head1=$(git rev-parse --short HEAD) &&
@@ -295,13 +295,13 @@ test_expect_success "Recursion picks up all submodules when necessary" '
 	echo "From $pwd/." > expect.err.2 &&
 	echo "   $head1..$head2  master     -> origin/master" >> expect.err.2 &&
 	cat expect.err.sub >> expect.err.2 &&
-	tail -2 expect.err >> expect.err.2 &&
+	tail -3 expect.err >> expect.err.2 &&
 	(
 		cd downstream &&
 		git fetch >../actual.out 2>../actual.err
 	) &&
 	test_i18ncmp expect.err.2 actual.err &&
-	test_i18ncmp expect.out actual.out
+	test_must_be_empty actual.out
 '
 
 test_expect_success "'--recurse-submodules=on-demand' doesn't recurse when no new commits are fetched in the superproject (and ignores config)" '
@@ -317,7 +317,8 @@ test_expect_success "'--recurse-submodules=on-demand' doesn't recurse when no ne
 		git add subdir/deepsubmodule &&
 		git commit -m "new deepsubmodule" &&
 		head2=$(git rev-parse --short HEAD) &&
-		echo "From $pwd/submodule" > ../expect.err.sub &&
+		echo Fetching submodule submodule > ../expect.err.sub &&
+		echo "From $pwd/submodule" >> ../expect.err.sub &&
 		echo "   $head1..$head2  master     -> origin/master" >> ../expect.err.sub
 	) &&
 	(
@@ -335,7 +336,7 @@ test_expect_success "'--recurse-submodules=on-demand' recurses as deep as necess
 	git add submodule &&
 	git commit -m "new submodule" &&
 	head2=$(git rev-parse --short HEAD) &&
-	tail -2 expect.err > expect.err.deepsub &&
+	tail -3 expect.err > expect.err.deepsub &&
 	echo "From $pwd/." > expect.err &&
 	echo "   $head1..$head2  master     -> origin/master" >>expect.err &&
 	cat expect.err.sub >> expect.err &&
@@ -354,7 +355,7 @@ test_expect_success "'--recurse-submodules=on-demand' recurses as deep as necess
 			git config --unset -f .gitmodules submodule.subdir/deepsubmodule.fetchRecursive
 		)
 	) &&
-	test_i18ncmp expect.out actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err actual.err
 '
 
@@ -388,7 +389,7 @@ test_expect_success "'fetch.recurseSubmodules=on-demand' overrides global config
 	head2=$(git rev-parse --short HEAD) &&
 	echo "From $pwd/." > expect.err.2 &&
 	echo "   $head1..$head2  master     -> origin/master" >>expect.err.2 &&
-	head -2 expect.err >> expect.err.2 &&
+	head -3 expect.err >> expect.err.2 &&
 	(
 		cd downstream &&
 		git config fetch.recurseSubmodules on-demand &&
@@ -399,7 +400,7 @@ test_expect_success "'fetch.recurseSubmodules=on-demand' overrides global config
 		cd downstream &&
 		git config --unset fetch.recurseSubmodules
 	) &&
-	test_i18ncmp expect.out.sub actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err.2 actual.err
 '
 
@@ -416,7 +417,7 @@ test_expect_success "'submodule.<sub>.fetchRecurseSubmodules=on-demand' override
 	head2=$(git rev-parse --short HEAD) &&
 	echo "From $pwd/." > expect.err.2 &&
 	echo "   $head1..$head2  master     -> origin/master" >>expect.err.2 &&
-	head -2 expect.err >> expect.err.2 &&
+	head -3 expect.err >> expect.err.2 &&
 	(
 		cd downstream &&
 		git config submodule.submodule.fetchRecurseSubmodules on-demand &&
@@ -427,7 +428,7 @@ test_expect_success "'submodule.<sub>.fetchRecurseSubmodules=on-demand' override
 		cd downstream &&
 		git config --unset submodule.submodule.fetchRecurseSubmodules
 	) &&
-	test_i18ncmp expect.out.sub actual.out &&
+	test_must_be_empty actual.out &&
 	test_i18ncmp expect.err.2 actual.err
 '
 
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 02/14] xread: poll on non blocking fds
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 01/14] submodule: Send "Fetching submodule <foo>" to standard error Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 03/14] xread_nonblock: add functionality to read from fds without blocking Stefan Beller
                   ` (11 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

>From the man page:
EAGAIN The file descriptor fd refers to a file other than a socket
       and has been marked nonblocking (O_NONBLOCK), and the read
       would block.

EAGAIN or EWOULDBLOCK
       The file descriptor fd refers to a socket and has been marked
       nonblocking (O_NONBLOCK), and the read would block.  POSIX.1-2001
       allows either error to be returned for this case, and does not
       require these constants to have the same value, so a portable
       application should check for both possibilities.

If we get an EAGAIN or EWOULDBLOCK the fd must have set O_NONBLOCK.
As the intent of xread is to read as much as possible either until the
fd is EOF or an actual error occurs, we can ease the feeder of the fd
by not spinning the whole time, but rather wait for it politely by not
busy waiting.

We should not care if the call to poll failed, as we're in an infinite
loop and can only get out with the correct read().

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 wrapper.c | 13 +++++++++++--
 1 file changed, 11 insertions(+), 2 deletions(-)

diff --git a/wrapper.c b/wrapper.c
index ff49807..5517928 100644
--- a/wrapper.c
+++ b/wrapper.c
@@ -201,8 +201,17 @@ ssize_t xread(int fd, void *buf, size_t len)
 	    len = MAX_IO_SIZE;
 	while (1) {
 		nr = read(fd, buf, len);
-		if ((nr < 0) && (errno == EAGAIN || errno == EINTR))
-			continue;
+		if (nr < 0) {
+			if (errno == EINTR)
+				continue;
+			if (errno == EAGAIN || errno == EWOULDBLOCK) {
+				struct pollfd pfd;
+				pfd.events = POLLIN;
+				pfd.fd = fd;
+				/* We deliberately ignore the return value */
+				poll(&pfd, 1, -1);
+			}
+		}
 		return nr;
 	}
 }
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 03/14] xread_nonblock: add functionality to read from fds without blocking
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 01/14] submodule: Send "Fetching submodule <foo>" to standard error Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 02/14] xread: poll on non blocking fds Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 04/14] strbuf: add strbuf_read_once to read " Stefan Beller
                   ` (10 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Provide a wrapper to read(), similar to xread(), that restarts on
EINTR but not EAGAIN (or EWOULDBLOCK). This enables the caller to
handle polling itself, possibly polling multiple sockets or performing
some other action.

Helped-by: Jacob Keller <jacob.keller@gmail.com>
Helped-by: Jeff King <peff@peff.net>,
Helped-by: Junio C Hamano <gitster@pobox.com>
Signed-off-by: Stefan Beller <sbeller@google.com>
---
 git-compat-util.h |  1 +
 wrapper.c         | 22 ++++++++++++++++++++++
 2 files changed, 23 insertions(+)

diff --git a/git-compat-util.h b/git-compat-util.h
index c6d391f..9ccea85 100644
--- a/git-compat-util.h
+++ b/git-compat-util.h
@@ -718,6 +718,7 @@ extern void *xcalloc(size_t nmemb, size_t size);
 extern void *xmmap(void *start, size_t length, int prot, int flags, int fd, off_t offset);
 extern void *xmmap_gently(void *start, size_t length, int prot, int flags, int fd, off_t offset);
 extern ssize_t xread(int fd, void *buf, size_t len);
+extern ssize_t xread_nonblock(int fd, void *buf, size_t len);
 extern ssize_t xwrite(int fd, const void *buf, size_t len);
 extern ssize_t xpread(int fd, void *buf, size_t len, off_t offset);
 extern int xdup(int fd);
diff --git a/wrapper.c b/wrapper.c
index 5517928..41a21e1 100644
--- a/wrapper.c
+++ b/wrapper.c
@@ -217,6 +217,28 @@ ssize_t xread(int fd, void *buf, size_t len)
 }
 
 /*
+ * xread_nonblock() is the same a read(), but it automatically restarts read()
+ * interrupted operations (EINTR). xread_nonblock() DOES NOT GUARANTEE that
+ * "len" bytes is read. EWOULDBLOCK is turned into EAGAIN.
+ */
+ssize_t xread_nonblock(int fd, void *buf, size_t len)
+{
+	ssize_t nr;
+	if (len > MAX_IO_SIZE)
+		len = MAX_IO_SIZE;
+	while (1) {
+		nr = read(fd, buf, len);
+		if (nr < 0) {
+			if (errno == EINTR)
+				continue;
+			if (errno == EWOULDBLOCK)
+				errno = EAGAIN;
+		}
+		return nr;
+	}
+}
+
+/*
  * xwrite() is the same a write(), but it automatically restarts write()
  * operations with a recoverable error (EAGAIN and EINTR). xwrite() DOES NOT
  * GUARANTEE that "len" bytes is written even if the operation is successful.
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 04/14] strbuf: add strbuf_read_once to read without blocking
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (2 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 03/14] xread_nonblock: add functionality to read from fds without blocking Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 05/14] run-command: factor out return value computation Stefan Beller
                   ` (9 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

The new call will read from a file descriptor into a strbuf once. The
underlying call xread_nonblock is meant to execute without blocking if
the file descriptor is set to O_NONBLOCK. It is a bug to call
strbuf_read_once on a file descriptor which would block.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 strbuf.c | 11 +++++++++++
 strbuf.h |  9 +++++++++
 2 files changed, 20 insertions(+)

diff --git a/strbuf.c b/strbuf.c
index cce5eed..35e71b8 100644
--- a/strbuf.c
+++ b/strbuf.c
@@ -384,6 +384,17 @@ ssize_t strbuf_read(struct strbuf *sb, int fd, size_t hint)
 	return sb->len - oldlen;
 }
 
+ssize_t strbuf_read_once(struct strbuf *sb, int fd, size_t hint)
+{
+	ssize_t cnt;
+
+	strbuf_grow(sb, hint ? hint : 8192);
+	cnt = xread_nonblock(fd, sb->buf + sb->len, sb->alloc - sb->len - 1);
+	if (cnt > 0)
+		strbuf_setlen(sb, sb->len + cnt);
+	return cnt;
+}
+
 #define STRBUF_MAXLINK (2*PATH_MAX)
 
 int strbuf_readlink(struct strbuf *sb, const char *path, size_t hint)
diff --git a/strbuf.h b/strbuf.h
index aef2794..ea69665 100644
--- a/strbuf.h
+++ b/strbuf.h
@@ -367,6 +367,15 @@ extern size_t strbuf_fread(struct strbuf *, size_t, FILE *);
 extern ssize_t strbuf_read(struct strbuf *, int fd, size_t hint);
 
 /**
+ * Read from a file descriptor that is marked as O_NONBLOCK without
+ * blocking.  Returns the number of new bytes appended to the sb.
+ * Negative return value signals there was an error returned from
+ * underlying read(2), in which case the caller should check errno.
+ * e.g. errno == EAGAIN when the read may have blocked.
+ */
+extern ssize_t strbuf_read_once(struct strbuf *, int fd, size_t hint);
+
+/**
  * Read the contents of a file, specified by its path. The third argument
  * can be used to give a hint about the file size, to avoid reallocs.
  */
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 05/14] run-command: factor out return value computation
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (3 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 04/14] strbuf: add strbuf_read_once to read " Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 06/14] run-command: add an asynchronous parallel child processor Stefan Beller
                   ` (8 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

We will need computing the return value in a later patch without the
wait.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 run-command.c | 56 ++++++++++++++++++++++++++++++++++----------------------
 1 file changed, 34 insertions(+), 22 deletions(-)

diff --git a/run-command.c b/run-command.c
index 28e1d55..2bf1aba 100644
--- a/run-command.c
+++ b/run-command.c
@@ -232,6 +232,35 @@ static inline void set_cloexec(int fd)
 		fcntl(fd, F_SETFD, flags | FD_CLOEXEC);
 }
 
+static int determine_return_value(int wait_status,
+				  int *result,
+				  int *error_code,
+				  const char *argv0)
+{
+	if (WIFSIGNALED(wait_status)) {
+		*result = WTERMSIG(wait_status);
+		if (*result != SIGINT && *result != SIGQUIT)
+			error("%s died of signal %d", argv0, *result);
+		/*
+		 * This return value is chosen so that code & 0xff
+		 * mimics the exit code that a POSIX shell would report for
+		 * a program that died from this signal.
+		 */
+		*result += 128;
+	} else if (WIFEXITED(wait_status)) {
+		*result = WEXITSTATUS(wait_status);
+		/*
+		 * Convert special exit code when execvp failed.
+		 */
+		if (*result == 127) {
+			*result = -1;
+			*error_code = ENOENT;
+		}
+	} else
+		return -1;
+	return 0;
+}
+
 static int wait_or_whine(pid_t pid, const char *argv0)
 {
 	int status, code = -1;
@@ -244,29 +273,12 @@ static int wait_or_whine(pid_t pid, const char *argv0)
 	if (waiting < 0) {
 		failed_errno = errno;
 		error("waitpid for %s failed: %s", argv0, strerror(errno));
-	} else if (waiting != pid) {
-		error("waitpid is confused (%s)", argv0);
-	} else if (WIFSIGNALED(status)) {
-		code = WTERMSIG(status);
-		if (code != SIGINT && code != SIGQUIT)
-			error("%s died of signal %d", argv0, code);
-		/*
-		 * This return value is chosen so that code & 0xff
-		 * mimics the exit code that a POSIX shell would report for
-		 * a program that died from this signal.
-		 */
-		code += 128;
-	} else if (WIFEXITED(status)) {
-		code = WEXITSTATUS(status);
-		/*
-		 * Convert special exit code when execvp failed.
-		 */
-		if (code == 127) {
-			code = -1;
-			failed_errno = ENOENT;
-		}
 	} else {
-		error("waitpid is confused (%s)", argv0);
+		if (waiting != pid || (determine_return_value(status,
+							      &code,
+							      &failed_errno,
+							      argv0) < 0))
+			error("waitpid is confused (%s)", argv0);
 	}
 
 	clear_child_for_cleanup(pid);
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (4 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 05/14] run-command: factor out return value computation Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  6:29   ` Junio C Hamano
  2015-09-23  6:47   ` Junio C Hamano
  2015-09-23  1:45 ` [PATCHv4 07/14] fetch_populated_submodules: use new parallel job processing Stefan Beller
                   ` (7 subsequent siblings)
  13 siblings, 2 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

This allows to run external commands in parallel with ordered output
on stderr.

If we run external commands in parallel we cannot pipe the output directly
to the our stdout/err as it would mix up. So each process's output will
flow through a pipe, which we buffer. One subprocess can be directly
piped to out stdout/err for a low latency feedback to the user.

Example:
Let's assume we have 5 submodules A,B,C,D,E and each fetch takes a
different amount of time as the different submodules vary in size, then
the output of fetches in sequential order might look like this:

 time -->
 output: |---A---| |-B-| |-------C-------| |-D-| |-E-|

When we schedule these submodules into maximal two parallel processes,
a schedule and sample output over time may look like this:

process 1: |---A---| |-D-| |-E-|

process 2: |-B-| |-------C-------|

output:    |---A---|B|---C-------|DE

So A will be perceived as it would run normally in the single child
version. As B has finished by the time A is done, we can dump its whole
progress buffer on stderr, such that it looks like it finished in no
time. Once that is done, C is determined to be the visible child and
its progress will be reported in real time.

So this way of output is really good for human consumption, as it only
changes the timing, not the actual output.

For machine consumption the output needs to be prepared in the tasks,
by either having a prefix per line or per block to indicate whose tasks
output is displayed, because the output order may not follow the
original sequential ordering:

 |----A----| |--B--| |-C-|

will be scheduled to be all parallel:

process 1: |----A----|
process 2: |--B--|
process 3: |-C-|
output:    |----A----|CB

This happens because C finished before B did, so it will be queued for
output before B.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 run-command.c          | 264 +++++++++++++++++++++++++++++++++++++++++++++++++
 run-command.h          |  36 +++++++
 t/t0061-run-command.sh |  20 ++++
 test-run-command.c     |  24 +++++
 4 files changed, 344 insertions(+)

diff --git a/run-command.c b/run-command.c
index 2bf1aba..494e1f8 100644
--- a/run-command.c
+++ b/run-command.c
@@ -3,6 +3,8 @@
 #include "exec_cmd.h"
 #include "sigchain.h"
 #include "argv-array.h"
+#include "thread-utils.h"
+#include "strbuf.h"
 
 void child_process_init(struct child_process *child)
 {
@@ -864,3 +866,265 @@ int capture_command(struct child_process *cmd, struct strbuf *buf, size_t hint)
 	close(cmd->out);
 	return finish_command(cmd);
 }
+
+struct parallel_processes {
+	void *data;
+
+	int max_processes;
+	int nr_processes;
+
+	get_next_task_fn get_next_task;
+	start_failure_fn start_failure;
+	return_value_fn return_value;
+
+	struct {
+		unsigned in_use : 1;
+		struct child_process process;
+		struct strbuf err;
+	} *children;
+	/*
+	 * The struct pollfd is logically part of *children,
+	 * but the system call expects it as its own array.
+	 */
+	struct pollfd *pfd;
+
+	int output_owner;
+	struct strbuf buffered_output; /* of finished children */
+};
+
+void default_start_failure(void *data,
+			   struct child_process *cp,
+			   struct strbuf *err)
+{
+	int i;
+	struct strbuf sb = STRBUF_INIT;
+
+	for (i = 0; cp->argv[i]; i++)
+		strbuf_addf(&sb, " %s", cp->argv[i]);
+
+	die_errno("Starting a child failed:%s", sb.buf);
+}
+
+void default_return_value(void *data,
+			  struct child_process *cp,
+			  int result)
+{
+	int i;
+	struct strbuf sb = STRBUF_INIT;
+
+	if (!result)
+		return;
+
+	for (i = 0; cp->argv[i]; i++)
+		strbuf_addf(&sb, " %s", cp->argv[i]);
+
+	die_errno("A child failed with return code %d:%s", result, sb.buf);
+}
+
+static void pp_init(struct parallel_processes *pp,
+					int n, void *data,
+					get_next_task_fn get_next_task,
+					start_failure_fn start_failure,
+					return_value_fn return_value)
+{
+	int i;
+
+	if (n < 1)
+		n = online_cpus();
+
+	pp->max_processes = n;
+	pp->data = data;
+	if (!get_next_task)
+		die("BUG: you need to specify a get_next_task function");
+	pp->get_next_task = get_next_task;
+
+	pp->start_failure = start_failure ? start_failure : default_start_failure;
+	pp->return_value = return_value ? return_value : default_return_value;
+
+	pp->nr_processes = 0;
+	pp->output_owner = 0;
+	pp->children = xcalloc(n, sizeof(*pp->children));
+	pp->pfd = xcalloc(n, sizeof(*pp->pfd));
+	strbuf_init(&pp->buffered_output, 0);
+
+	for (i = 0; i < n; i++) {
+		strbuf_init(&pp->children[i].err, 0);
+		pp->pfd[i].events = POLLIN;
+		pp->pfd[i].fd = -1;
+	}
+}
+
+static void pp_cleanup(struct parallel_processes *pp)
+{
+	int i;
+
+	for (i = 0; i < pp->max_processes; i++)
+		strbuf_release(&pp->children[i].err);
+
+	free(pp->children);
+	free(pp->pfd);
+	strbuf_release(&pp->buffered_output);
+}
+
+static void set_nonblocking(int fd)
+{
+	int flags = fcntl(fd, F_GETFL);
+	if (flags < 0)
+		warning("Could not get file status flags, "
+			"output will be degraded");
+	else if (fcntl(fd, F_SETFL, flags | O_NONBLOCK))
+		warning("Could not set file status flags, "
+			"output will be degraded");
+}
+
+/* returns 1 if a process was started, 0 otherwise */
+static int pp_start_one(struct parallel_processes *pp)
+{
+	int i;
+
+	for (i = 0; i < pp->max_processes; i++)
+		if (!pp->children[i].in_use)
+			break;
+	if (i == pp->max_processes)
+		die("BUG: bookkeeping is hard");
+
+	if (!pp->get_next_task(pp->data,
+			       &pp->children[i].process,
+			       &pp->children[i].err))
+		return 1;
+
+	if (start_command(&pp->children[i].process))
+		pp->start_failure(pp->data,
+				  &pp->children[i].process,
+				  &pp->children[i].err);
+
+	set_nonblocking(pp->children[i].process.err);
+
+	pp->nr_processes++;
+	pp->children[i].in_use = 1;
+	pp->pfd[i].fd = pp->children[i].process.err;
+	return 0;
+}
+
+static void pp_buffer_stderr(struct parallel_processes *pp)
+{
+	int i;
+
+	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
+		if (errno == EINTR)
+			continue;
+		pp_cleanup(pp);
+		die_errno("poll");
+	}
+
+	/* Buffer output from all pipes. */
+	for (i = 0; i < pp->max_processes; i++) {
+		if (pp->children[i].in_use &&
+		    pp->pfd[i].revents & POLLIN)
+			if (strbuf_read_once(&pp->children[i].err,
+					     pp->children[i].process.err, 0) < 0)
+				if (errno != EAGAIN)
+					die_errno("read");
+	}
+}
+
+static void pp_output(struct parallel_processes *pp)
+{
+	int i = pp->output_owner;
+	if (pp->children[i].in_use &&
+	    pp->children[i].err.len) {
+		fputs(pp->children[i].err.buf, stderr);
+		strbuf_reset(&pp->children[i].err);
+	}
+}
+
+static void pp_collect_finished(struct parallel_processes *pp)
+{
+	int i = 0;
+	pid_t pid;
+	int wait_status, code;
+	int n = pp->max_processes;
+
+	while (pp->nr_processes > 0) {
+		pid = waitpid(-1, &wait_status, WNOHANG);
+		if (pid == 0)
+			return;
+
+		if (pid < 0)
+			die_errno("wait");
+
+		for (i = 0; i < pp->max_processes; i++)
+			if (pp->children[i].in_use &&
+			    pid == pp->children[i].process.pid)
+				break;
+		if (i == pp->max_processes)
+			die("BUG: found a child process we were not aware of");
+
+		if (strbuf_read(&pp->children[i].err,
+				pp->children[i].process.err, 0) < 0)
+			die_errno("strbuf_read");
+
+		if (determine_return_value(wait_status, &code, &errno,
+					   pp->children[i].process.argv[0]) < 0)
+			error("waitpid is confused (%s)",
+			      pp->children[i].process.argv[0]);
+
+		pp->return_value(pp->data, &pp->children[i].process, code);
+
+		argv_array_clear(&pp->children[i].process.args);
+		argv_array_clear(&pp->children[i].process.env_array);
+
+		pp->nr_processes--;
+		pp->children[i].in_use = 0;
+		pp->pfd[i].fd = -1;
+
+		if (i != pp->output_owner) {
+			strbuf_addbuf(&pp->buffered_output, &pp->children[i].err);
+			strbuf_reset(&pp->children[i].err);
+		} else {
+			fputs(pp->children[i].err.buf, stderr);
+			strbuf_reset(&pp->children[i].err);
+
+			/* Output all other finished child processes */
+			fputs(pp->buffered_output.buf, stderr);
+			strbuf_reset(&pp->buffered_output);
+
+			/*
+			 * Pick next process to output live.
+			 * NEEDSWORK:
+			 * For now we pick it randomly by doing a round
+			 * robin. Later we may want to pick the one with
+			 * the most output or the longest or shortest
+			 * running process time.
+			 */
+			for (i = 0; i < n; i++)
+				if (pp->children[(pp->output_owner + i) % n].in_use)
+					break;
+			pp->output_owner = (pp->output_owner + i) % n;
+		}
+	}
+}
+
+int run_processes_parallel(int n, void *data,
+			   get_next_task_fn get_next_task,
+			   start_failure_fn start_failure,
+			   return_value_fn return_value)
+{
+	struct parallel_processes pp;
+	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
+
+	while (1) {
+		while (pp.nr_processes < pp.max_processes &&
+		       !pp_start_one(&pp))
+			; /* nothing */
+		if (!pp.nr_processes)
+			break;
+		pp_buffer_stderr(&pp);
+		pp_output(&pp);
+		pp_collect_finished(&pp);
+	}
+
+	pp_cleanup(&pp);
+
+	return 0;
+}
diff --git a/run-command.h b/run-command.h
index 5b4425a..3807fd1 100644
--- a/run-command.h
+++ b/run-command.h
@@ -119,4 +119,40 @@ struct async {
 int start_async(struct async *async);
 int finish_async(struct async *async);
 
+/**
+ * This callback should initialize the child process and preload the
+ * error channel. The preloading of is useful if you want to have a message
+ * printed directly before the output of the child process.
+ * You MUST set stdout_to_stderr.
+ *
+ * Return 1 if the next child is ready to run.
+ * Return 0 if there are no more tasks to be processed.
+ */
+typedef int (*get_next_task_fn)(void *data,
+				struct child_process *cp,
+				struct strbuf *err);
+
+typedef void (*start_failure_fn)(void *data,
+				 struct child_process *cp,
+				 struct strbuf *err);
+
+typedef void (*return_value_fn)(void *data,
+				struct child_process *cp,
+				int result);
+
+/**
+ * Runs up to n processes at the same time. Whenever a process can be
+ * started, the callback `get_next_task` is called to obtain the data
+ * fed to the child process.
+ *
+ * The children started via this function run in parallel and their output
+ * to stderr is buffered, while one of the children will directly output
+ * to stderr.
+ */
+
+int run_processes_parallel(int n, void *data,
+			   get_next_task_fn,
+			   start_failure_fn,
+			   return_value_fn);
+
 #endif
diff --git a/t/t0061-run-command.sh b/t/t0061-run-command.sh
index 9acf628..49aa3db 100755
--- a/t/t0061-run-command.sh
+++ b/t/t0061-run-command.sh
@@ -47,4 +47,24 @@ test_expect_success POSIXPERM,SANITY 'unreadable directory in PATH' '
 	test_cmp expect actual
 '
 
+cat >expect <<-EOF
+preloaded output of a child
+Hello
+World
+preloaded output of a child
+Hello
+World
+preloaded output of a child
+Hello
+World
+preloaded output of a child
+Hello
+World
+EOF
+
+test_expect_success 'run_command runs in parallel' '
+	test-run-command run-command-parallel-4 sh -c "printf \"%s\n%s\n\" Hello World" 2>actual &&
+	test_cmp expect actual
+'
+
 test_done
diff --git a/test-run-command.c b/test-run-command.c
index 89c7de2..94c6eee 100644
--- a/test-run-command.c
+++ b/test-run-command.c
@@ -10,9 +10,29 @@
 
 #include "git-compat-util.h"
 #include "run-command.h"
+#include "argv-array.h"
+#include "strbuf.h"
 #include <string.h>
 #include <errno.h>
 
+static int number_callbacks;
+int parallel_next(void *data,
+		  struct child_process *cp,
+		  struct strbuf *err)
+{
+	struct child_process *d = data;
+	if (number_callbacks >= 4)
+		return 0;
+
+	argv_array_pushv(&cp->args, d->argv);
+	cp->stdout_to_stderr = 1;
+	cp->no_stdin = 1;
+	cp->err = -1;
+	strbuf_addf(err, "preloaded output of a child\n");
+	number_callbacks++;
+	return 1;
+}
+
 int main(int argc, char **argv)
 {
 	struct child_process proc = CHILD_PROCESS_INIT;
@@ -30,6 +50,10 @@ int main(int argc, char **argv)
 	if (!strcmp(argv[1], "run-command"))
 		exit(run_command(&proc));
 
+	if (!strcmp(argv[1], "run-command-parallel-4"))
+		exit(run_processes_parallel(4, &proc, parallel_next,
+					 NULL, NULL));
+
 	fprintf(stderr, "check usage\n");
 	return 1;
 }
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 07/14] fetch_populated_submodules: use new parallel job processing
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (5 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 06/14] run-command: add an asynchronous parallel child processor Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 08/14] submodules: allow parallel fetching, add tests and documentation Stefan Beller
                   ` (6 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

In a later patch we enable parallel processing of submodules, this
only adds the possibility for it. So this change should not change
any user facing behavior.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 submodule.c | 119 +++++++++++++++++++++++++++++++++++++++++++-----------------
 1 file changed, 85 insertions(+), 34 deletions(-)

diff --git a/submodule.c b/submodule.c
index 1d64e57..d7c7a6e 100644
--- a/submodule.c
+++ b/submodule.c
@@ -12,6 +12,7 @@
 #include "sha1-array.h"
 #include "argv-array.h"
 #include "blob.h"
+#include "thread-utils.h"
 
 static int config_fetch_recurse_submodules = RECURSE_SUBMODULES_ON_DEMAND;
 static struct string_list changed_submodule_paths;
@@ -615,37 +616,82 @@ static void calculate_changed_submodule_paths(void)
 	initialized_fetch_ref_tips = 0;
 }
 
+struct submodule_parallel_fetch {
+	int count;
+	struct argv_array args;
+	const char *work_tree;
+	const char *prefix;
+	int command_line_option;
+	int quiet;
+	int result;
+};
+#define SPF_INIT {0, ARGV_ARRAY_INIT, NULL, NULL, 0, 0, 0}
+
+int get_next_submodule(void *data, struct child_process *cp,
+		       struct strbuf *err);
+
+void handle_submodule_fetch_start_err(void *data, struct child_process *cp, struct strbuf *err)
+{
+	struct submodule_parallel_fetch *spf = data;
+	spf->result = 1;
+}
+
+void handle_submodule_fetch_finish( void *data, struct child_process *cp, int retvalue)
+{
+	struct submodule_parallel_fetch *spf = data;
+
+	if (retvalue)
+		spf->result = 1;
+}
+
 int fetch_populated_submodules(const struct argv_array *options,
 			       const char *prefix, int command_line_option,
 			       int quiet)
 {
-	int i, result = 0;
-	struct child_process cp = CHILD_PROCESS_INIT;
-	struct argv_array argv = ARGV_ARRAY_INIT;
-	const char *work_tree = get_git_work_tree();
-	if (!work_tree)
+	int i;
+	int max_parallel_jobs = 1;
+	struct submodule_parallel_fetch spf = SPF_INIT;
+
+	spf.work_tree = get_git_work_tree();
+	spf.command_line_option = command_line_option;
+	spf.quiet = quiet;
+	spf.prefix = prefix;
+
+	if (!spf.work_tree)
 		goto out;
 
 	if (read_cache() < 0)
 		die("index file corrupt");
 
-	argv_array_push(&argv, "fetch");
+	argv_array_push(&spf.args, "fetch");
 	for (i = 0; i < options->argc; i++)
-		argv_array_push(&argv, options->argv[i]);
-	argv_array_push(&argv, "--recurse-submodules-default");
+		argv_array_push(&spf.args, options->argv[i]);
+	argv_array_push(&spf.args, "--recurse-submodules-default");
 	/* default value, "--submodule-prefix" and its value are added later */
 
-	cp.env = local_repo_env;
-	cp.git_cmd = 1;
-	cp.no_stdin = 1;
-
 	calculate_changed_submodule_paths();
+	run_processes_parallel(max_parallel_jobs, &spf,
+			       get_next_submodule,
+			       handle_submodule_fetch_start_err,
+			       handle_submodule_fetch_finish);
+
+	argv_array_clear(&spf.args);
+out:
+	string_list_clear(&changed_submodule_paths, 1);
+	return spf.result;
+}
+
+int get_next_submodule(void *data, struct child_process *cp,
+		       struct strbuf *err)
+{
+	int ret = 0;
+	struct submodule_parallel_fetch *spf = data;
 
-	for (i = 0; i < active_nr; i++) {
+	for ( ; spf->count < active_nr; spf->count++) {
 		struct strbuf submodule_path = STRBUF_INIT;
 		struct strbuf submodule_git_dir = STRBUF_INIT;
 		struct strbuf submodule_prefix = STRBUF_INIT;
-		const struct cache_entry *ce = active_cache[i];
+		const struct cache_entry *ce = active_cache[spf->count];
 		const char *git_dir, *default_argv;
 		const struct submodule *submodule;
 
@@ -657,7 +703,7 @@ int fetch_populated_submodules(const struct argv_array *options,
 			submodule = submodule_from_name(null_sha1, ce->name);
 
 		default_argv = "yes";
-		if (command_line_option == RECURSE_SUBMODULES_DEFAULT) {
+		if (spf->command_line_option == RECURSE_SUBMODULES_DEFAULT) {
 			if (submodule &&
 			    submodule->fetch_recurse !=
 						RECURSE_SUBMODULES_NONE) {
@@ -680,40 +726,45 @@ int fetch_populated_submodules(const struct argv_array *options,
 					default_argv = "on-demand";
 				}
 			}
-		} else if (command_line_option == RECURSE_SUBMODULES_ON_DEMAND) {
+		} else if (spf->command_line_option == RECURSE_SUBMODULES_ON_DEMAND) {
 			if (!unsorted_string_list_lookup(&changed_submodule_paths, ce->name))
 				continue;
 			default_argv = "on-demand";
 		}
 
-		strbuf_addf(&submodule_path, "%s/%s", work_tree, ce->name);
+		strbuf_addf(&submodule_path, "%s/%s", spf->work_tree, ce->name);
 		strbuf_addf(&submodule_git_dir, "%s/.git", submodule_path.buf);
-		strbuf_addf(&submodule_prefix, "%s%s/", prefix, ce->name);
+		strbuf_addf(&submodule_prefix, "%s%s/", spf->prefix, ce->name);
 		git_dir = read_gitfile(submodule_git_dir.buf);
 		if (!git_dir)
 			git_dir = submodule_git_dir.buf;
 		if (is_directory(git_dir)) {
-			if (!quiet)
-				fprintf(stderr, "Fetching submodule %s%s\n", prefix, ce->name);
-			cp.dir = submodule_path.buf;
-			argv_array_push(&argv, default_argv);
-			argv_array_push(&argv, "--submodule-prefix");
-			argv_array_push(&argv, submodule_prefix.buf);
-			cp.argv = argv.argv;
-			if (run_command(&cp))
-				result = 1;
-			argv_array_pop(&argv);
-			argv_array_pop(&argv);
-			argv_array_pop(&argv);
+			child_process_init(cp);
+			cp->dir = strbuf_detach(&submodule_path, NULL);
+			cp->env = local_repo_env;
+			cp->git_cmd = 1;
+			cp->no_stdin = 1;
+			cp->stdout_to_stderr = 1;
+			cp->err = -1;
+			if (!spf->quiet)
+				strbuf_addf(err, "Fetching submodule %s%s\n",
+					    spf->prefix, ce->name);
+			argv_array_init(&cp->args);
+			argv_array_pushv(&cp->args, spf->args.argv);
+			argv_array_push(&cp->args, default_argv);
+			argv_array_push(&cp->args, "--submodule-prefix");
+			argv_array_push(&cp->args, submodule_prefix.buf);
+			ret = 1;
 		}
 		strbuf_release(&submodule_path);
 		strbuf_release(&submodule_git_dir);
 		strbuf_release(&submodule_prefix);
+		if (ret) {
+			spf->count++;
+			return 1;
+		}
 	}
-	argv_array_clear(&argv);
-out:
-	string_list_clear(&changed_submodule_paths, 1);
-	return result;
+	return 0;
 }
 
 unsigned is_submodule_modified(const char *path, int ignore_untracked)
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 08/14] submodules: allow parallel fetching, add tests and documentation
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (6 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 07/14] fetch_populated_submodules: use new parallel job processing Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 09/14] submodule-config: Untangle logic in parse_config Stefan Beller
                   ` (5 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

This enables the work of the previous patches.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 Documentation/fetch-options.txt |  7 +++++++
 builtin/fetch.c                 |  6 +++++-
 builtin/pull.c                  |  6 ++++++
 submodule.c                     |  3 +--
 submodule.h                     |  2 +-
 t/t5526-fetch-submodules.sh     | 19 +++++++++++++++++++
 6 files changed, 39 insertions(+), 4 deletions(-)

diff --git a/Documentation/fetch-options.txt b/Documentation/fetch-options.txt
index 45583d8..6b109f6 100644
--- a/Documentation/fetch-options.txt
+++ b/Documentation/fetch-options.txt
@@ -100,6 +100,13 @@ ifndef::git-pull[]
 	reference to a commit that isn't already in the local submodule
 	clone.
 
+-j::
+--jobs=<n>::
+	Number of parallel children to be used for fetching submodules.
+	Each will fetch from different submodules, such that fetching many
+	submodules will be faster. By default submodules will be fetched
+	one at a time.
+
 --no-recurse-submodules::
 	Disable recursive fetching of submodules (this has the same effect as
 	using the '--recurse-submodules=no' option).
diff --git a/builtin/fetch.c b/builtin/fetch.c
index ee1f1a9..f28eac6 100644
--- a/builtin/fetch.c
+++ b/builtin/fetch.c
@@ -37,6 +37,7 @@ static int prune = -1; /* unspecified */
 static int all, append, dry_run, force, keep, multiple, update_head_ok, verbosity;
 static int progress = -1, recurse_submodules = RECURSE_SUBMODULES_DEFAULT;
 static int tags = TAGS_DEFAULT, unshallow, update_shallow;
+static int max_children = 1;
 static const char *depth;
 static const char *upload_pack;
 static struct strbuf default_rla = STRBUF_INIT;
@@ -99,6 +100,8 @@ static struct option builtin_fetch_options[] = {
 		    N_("fetch all tags and associated objects"), TAGS_SET),
 	OPT_SET_INT('n', NULL, &tags,
 		    N_("do not fetch all tags (--no-tags)"), TAGS_UNSET),
+	OPT_INTEGER('j', "jobs", &max_children,
+		    N_("number of submodules fetched in parallel")),
 	OPT_BOOL('p', "prune", &prune,
 		 N_("prune remote-tracking branches no longer on remote")),
 	{ OPTION_CALLBACK, 0, "recurse-submodules", NULL, N_("on-demand"),
@@ -1217,7 +1220,8 @@ int cmd_fetch(int argc, const char **argv, const char *prefix)
 		result = fetch_populated_submodules(&options,
 						    submodule_prefix,
 						    recurse_submodules,
-						    verbosity < 0);
+						    verbosity < 0,
+						    max_children);
 		argv_array_clear(&options);
 	}
 
diff --git a/builtin/pull.c b/builtin/pull.c
index 722a83c..f0af196 100644
--- a/builtin/pull.c
+++ b/builtin/pull.c
@@ -94,6 +94,7 @@ static int opt_force;
 static char *opt_tags;
 static char *opt_prune;
 static char *opt_recurse_submodules;
+static char *max_children;
 static int opt_dry_run;
 static char *opt_keep;
 static char *opt_depth;
@@ -177,6 +178,9 @@ static struct option pull_options[] = {
 		N_("on-demand"),
 		N_("control recursive fetching of submodules"),
 		PARSE_OPT_OPTARG),
+	OPT_PASSTHRU('j', "jobs", &max_children, N_("n"),
+		N_("number of submodules pulled in parallel"),
+		PARSE_OPT_OPTARG),
 	OPT_BOOL(0, "dry-run", &opt_dry_run,
 		N_("dry run")),
 	OPT_PASSTHRU('k', "keep", &opt_keep, NULL,
@@ -524,6 +528,8 @@ static int run_fetch(const char *repo, const char **refspecs)
 		argv_array_push(&args, opt_prune);
 	if (opt_recurse_submodules)
 		argv_array_push(&args, opt_recurse_submodules);
+	if (max_children)
+		argv_array_push(&args, max_children);
 	if (opt_dry_run)
 		argv_array_push(&args, "--dry-run");
 	if (opt_keep)
diff --git a/submodule.c b/submodule.c
index d7c7a6e..fdaf3e4 100644
--- a/submodule.c
+++ b/submodule.c
@@ -646,10 +646,9 @@ void handle_submodule_fetch_finish( void *data, struct child_process *cp, int re
 
 int fetch_populated_submodules(const struct argv_array *options,
 			       const char *prefix, int command_line_option,
-			       int quiet)
+			       int quiet, int max_parallel_jobs)
 {
 	int i;
-	int max_parallel_jobs = 1;
 	struct submodule_parallel_fetch spf = SPF_INIT;
 
 	spf.work_tree = get_git_work_tree();
diff --git a/submodule.h b/submodule.h
index 5507c3d..cbc0003 100644
--- a/submodule.h
+++ b/submodule.h
@@ -31,7 +31,7 @@ void set_config_fetch_recurse_submodules(int value);
 void check_for_new_submodule_commits(unsigned char new_sha1[20]);
 int fetch_populated_submodules(const struct argv_array *options,
 			       const char *prefix, int command_line_option,
-			       int quiet);
+			       int quiet, int max_parallel_jobs);
 unsigned is_submodule_modified(const char *path, int ignore_untracked);
 int submodule_uses_gitfile(const char *path);
 int ok_to_remove_submodule(const char *path);
diff --git a/t/t5526-fetch-submodules.sh b/t/t5526-fetch-submodules.sh
index 17759b1..1b4ce69 100755
--- a/t/t5526-fetch-submodules.sh
+++ b/t/t5526-fetch-submodules.sh
@@ -71,6 +71,16 @@ test_expect_success "fetch --recurse-submodules recurses into submodules" '
 	test_i18ncmp expect.err actual.err
 '
 
+test_expect_success "fetch --recurse-submodules -j2 has the same output behaviour" '
+	add_upstream_commit &&
+	(
+		cd downstream &&
+		git fetch --recurse-submodules -j2 2>../actual.err
+	) &&
+	test_must_be_empty actual.out &&
+	test_i18ncmp expect.err actual.err
+'
+
 test_expect_success "fetch alone only fetches superproject" '
 	add_upstream_commit &&
 	(
@@ -140,6 +150,15 @@ test_expect_success "--quiet propagates to submodules" '
 	! test -s actual.err
 '
 
+test_expect_success "--quiet propagates to parallel submodules" '
+	(
+		cd downstream &&
+		git fetch --recurse-submodules -j 2 --quiet  >../actual.out 2>../actual.err
+	) &&
+	! test -s actual.out &&
+	! test -s actual.err
+'
+
 test_expect_success "--dry-run propagates to submodules" '
 	add_upstream_commit &&
 	(
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 09/14] submodule-config: Untangle logic in parse_config
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (7 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 08/14] submodules: allow parallel fetching, add tests and documentation Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 10/14] submodule config: keep update strategy around Stefan Beller
                   ` (4 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

CC: Eric Sunshine <sunshine@sunshineco.com>
Signed-off-by: Stefan Beller <sbeller@google.com>
---
 submodule-config.c | 74 +++++++++++++++++++++---------------------------------
 1 file changed, 29 insertions(+), 45 deletions(-)

diff --git a/submodule-config.c b/submodule-config.c
index 393de53..afe0ea8 100644
--- a/submodule-config.c
+++ b/submodule-config.c
@@ -257,78 +257,62 @@ static int parse_config(const char *var, const char *value, void *data)
 	if (!name_and_item_from_var(var, &name, &item))
 		return 0;
 
-	submodule = lookup_or_create_by_name(me->cache, me->gitmodules_sha1,
-			name.buf);
+	submodule = lookup_or_create_by_name(me->cache,
+					     me->gitmodules_sha1,
+					     name.buf);
 
 	if (!strcmp(item.buf, "path")) {
-		struct strbuf path = STRBUF_INIT;
-		if (!value) {
+		if (!value)
 			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (!me->overwrite && submodule->path != NULL) {
+		else if (!me->overwrite && submodule->path != NULL)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"path");
-			goto release_return;
+		else {
+			if (submodule->path)
+				cache_remove_path(me->cache, submodule);
+			free((void *) submodule->path);
+			submodule->path = xstrdup(value);
+			cache_put_path(me->cache, submodule);
 		}
-
-		if (submodule->path)
-			cache_remove_path(me->cache, submodule);
-		free((void *) submodule->path);
-		strbuf_addstr(&path, value);
-		submodule->path = strbuf_detach(&path, NULL);
-		cache_put_path(me->cache, submodule);
 	} else if (!strcmp(item.buf, "fetchrecursesubmodules")) {
 		/* when parsing worktree configurations we can die early */
 		int die_on_error = is_null_sha1(me->gitmodules_sha1);
 		if (!me->overwrite &&
-		    submodule->fetch_recurse != RECURSE_SUBMODULES_NONE) {
+		    submodule->fetch_recurse != RECURSE_SUBMODULES_NONE)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"fetchrecursesubmodules");
-			goto release_return;
-		}
-
-		submodule->fetch_recurse = parse_fetch_recurse(var, value,
+		else
+			submodule->fetch_recurse = parse_fetch_recurse(
+								var, value,
 								die_on_error);
 	} else if (!strcmp(item.buf, "ignore")) {
-		struct strbuf ignore = STRBUF_INIT;
-		if (!me->overwrite && submodule->ignore != NULL) {
+		if (!value)
+			ret = config_error_nonbool(var);
+		else if (!me->overwrite && submodule->ignore != NULL)
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"ignore");
-			goto release_return;
-		}
-		if (!value) {
-			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (strcmp(value, "untracked") && strcmp(value, "dirty") &&
-		    strcmp(value, "all") && strcmp(value, "none")) {
+		else if (strcmp(value, "untracked") &&
+			 strcmp(value, "dirty") &&
+			 strcmp(value, "all") &&
+			 strcmp(value, "none"))
 			warning("Invalid parameter '%s' for config option "
 					"'submodule.%s.ignore'", value, var);
-			goto release_return;
+		else {
+			free((void *) submodule->ignore);
+			submodule->ignore = xstrdup(value);
 		}
-
-		free((void *) submodule->ignore);
-		strbuf_addstr(&ignore, value);
-		submodule->ignore = strbuf_detach(&ignore, NULL);
 	} else if (!strcmp(item.buf, "url")) {
-		struct strbuf url = STRBUF_INIT;
 		if (!value) {
 			ret = config_error_nonbool(var);
-			goto release_return;
-		}
-		if (!me->overwrite && submodule->url != NULL) {
+		} else if (!me->overwrite && submodule->url != NULL) {
 			warn_multiple_config(me->commit_sha1, submodule->name,
 					"url");
-			goto release_return;
+		} else {
+			free((void *) submodule->url);
+			submodule->url = xstrdup(value);
 		}
-
-		free((void *) submodule->url);
-		strbuf_addstr(&url, value);
-		submodule->url = strbuf_detach(&url, NULL);
 	}
 
-release_return:
 	strbuf_release(&name);
 	strbuf_release(&item);
 
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 10/14] submodule config: keep update strategy around
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (8 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 09/14] submodule-config: Untangle logic in parse_config Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 11/14] git submodule update: cmd_update_recursive Stefan Beller
                   ` (3 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

We need the submodule update strategies in a later patch.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 submodule-config.c | 11 +++++++++++
 submodule-config.h |  1 +
 2 files changed, 12 insertions(+)

diff --git a/submodule-config.c b/submodule-config.c
index afe0ea8..8b8c7d1 100644
--- a/submodule-config.c
+++ b/submodule-config.c
@@ -194,6 +194,7 @@ static struct submodule *lookup_or_create_by_name(struct submodule_cache *cache,
 
 	submodule->path = NULL;
 	submodule->url = NULL;
+	submodule->update = NULL;
 	submodule->fetch_recurse = RECURSE_SUBMODULES_NONE;
 	submodule->ignore = NULL;
 
@@ -311,6 +312,16 @@ static int parse_config(const char *var, const char *value, void *data)
 			free((void *) submodule->url);
 			submodule->url = xstrdup(value);
 		}
+	} else if (!strcmp(item.buf, "update")) {
+		if (!value)
+			ret = config_error_nonbool(var);
+		else if (!me->overwrite && submodule->update != NULL)
+			warn_multiple_config(me->commit_sha1, submodule->name,
+					     "update");
+		else {
+			free((void *)submodule->update);
+			submodule->update = xstrdup(value);
+		}
 	}
 
 	strbuf_release(&name);
diff --git a/submodule-config.h b/submodule-config.h
index 9061e4e..f9e2a29 100644
--- a/submodule-config.h
+++ b/submodule-config.h
@@ -14,6 +14,7 @@ struct submodule {
 	const char *url;
 	int fetch_recurse;
 	const char *ignore;
+	const char *update;
 	/* the sha1 blob id of the responsible .gitmodules file */
 	unsigned char gitmodules_sha1[20];
 };
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 11/14] git submodule update: cmd_update_recursive
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (9 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 10/14] submodule config: keep update strategy around Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 12/14] git submodule update: cmd_update_clone Stefan Beller
                   ` (2 subsequent siblings)
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Split the recursion part out to its own function, this allow us
in a later patch to convert cmd_update in C.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 git-submodule.sh | 47 ++++++++++++++++++++++++++---------------------
 1 file changed, 26 insertions(+), 21 deletions(-)

diff --git a/git-submodule.sh b/git-submodule.sh
index 8b0eb9a..ea3260e 100755
--- a/git-submodule.sh
+++ b/git-submodule.sh
@@ -582,6 +582,31 @@ cmd_deinit()
 	done
 }
 
+
+cmd_update_recursive()
+{
+	if test -n "$recursive"
+	then
+		(
+			prefix="$prefix$sm_path/"
+			clear_local_git_env
+			cd "$sm_path" &&
+			eval cmd_update
+		)
+		res=$?
+		if test $res -gt 0
+		then
+			die_msg="$(eval_gettext "Failed to recurse into submodule path '\$displaypath'")"
+			if test $res -eq 1
+			then
+				err="${err};$die_msg"
+			else
+				die_with_status $res "$die_msg"
+			fi
+		fi
+	fi
+}
+
 #
 # Update each submodule path to correct revision, using clone and checkout as needed
 #
@@ -790,27 +815,7 @@ Maybe you want to use 'update --init'?")"
 			fi
 		fi
 
-		if test -n "$recursive"
-		then
-			(
-				prefix="$prefix$sm_path/"
-				clear_local_git_env
-				cd "$sm_path" &&
-				eval cmd_update
-			)
-			res=$?
-			if test $res -gt 0
-			then
-				die_msg="$(eval_gettext "Failed to recurse into submodule path '\$displaypath'")"
-				if test $res -eq 1
-				then
-					err="${err};$die_msg"
-					continue
-				else
-					die_with_status $res "$die_msg"
-				fi
-			fi
-		fi
+		cmd_update_recursive
 	done
 
 	if test -n "$err"
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 12/14] git submodule update: cmd_update_clone
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (10 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 11/14] git submodule update: cmd_update_recursive Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23 20:13   ` Junio C Hamano
  2015-09-23  1:45 ` [PATCHv4 13/14] git submodule update: cmd_update_fetch Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 14/14] Rewrite submodule update in C Stefan Beller
  13 siblings, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Split the cloning part out to its own function,
this allow us in a later patch to convert cmd_update in C.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 git-submodule.sh | 31 ++++++++++++++++++++-----------
 1 file changed, 20 insertions(+), 11 deletions(-)

diff --git a/git-submodule.sh b/git-submodule.sh
index ea3260e..7f11158 100755
--- a/git-submodule.sh
+++ b/git-submodule.sh
@@ -607,6 +607,24 @@ cmd_update_recursive()
 	fi
 }
 
+cmd_update_clone()
+{
+	command="git checkout $subforce -q"
+	die_msg="$(eval_gettext "Unable to checkout '\$sha1' in submodule path '\$displaypath'")"
+	say_msg="$(eval_gettext "Submodule path '\$displaypath': checked out '\$sha1'")"
+
+	git submodule--helper clone ${GIT_QUIET:+--quiet} ${prefix:+--prefix "$prefix"} --path "$sm_path" --name "$name" --url "$url" "$reference" "$depth" || exit
+
+	if (clear_local_git_env; cd "$sm_path" && $command "$sha1")
+	then
+		say "$say_msg"
+	else
+		err="${err};$die_msg"
+		return
+	fi
+	cmd_update_recursive
+}
+
 #
 # Update each submodule path to correct revision, using clone and checkout as needed
 #
@@ -680,7 +698,6 @@ cmd_update()
 		cmd_init "--" "$@" || return
 	fi
 
-	cloned_modules=
 	git submodule--helper list --prefix "$wt_prefix" "$@" | {
 	err=
 	while read mode sha1 stage sm_path
@@ -725,9 +742,8 @@ Maybe you want to use 'update --init'?")"
 
 		if ! test -d "$sm_path"/.git && ! test -f "$sm_path"/.git
 		then
-			git submodule--helper clone ${GIT_QUIET:+--quiet} --prefix "$prefix" --path "$sm_path" --name "$name" --url "$url" "$reference" "$depth" || exit
-			cloned_modules="$cloned_modules;$name"
-			subsha1=
+			cmd_update_clone
+			continue
 		else
 			subsha1=$(clear_local_git_env; cd "$sm_path" &&
 				git rev-parse --verify HEAD) ||
@@ -767,13 +783,6 @@ Maybe you want to use 'update --init'?")"
 				die "$(eval_gettext "Unable to fetch in submodule path '\$displaypath'")"
 			fi
 
-			# Is this something we just cloned?
-			case ";$cloned_modules;" in
-			*";$name;"*)
-				# then there is no local change to integrate
-				update_module=checkout ;;
-			esac
-
 			must_die_on_failure=
 			case "$update_module" in
 			checkout)
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 13/14] git submodule update: cmd_update_fetch
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (11 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 12/14] git submodule update: cmd_update_clone Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  2015-09-23  1:45 ` [PATCHv4 14/14] Rewrite submodule update in C Stefan Beller
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Split the fetching part out to its own function,
this allow us in a later patch to convert cmd_update in C.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 git-submodule.sh | 164 ++++++++++++++++++++++++++++---------------------------
 1 file changed, 84 insertions(+), 80 deletions(-)

diff --git a/git-submodule.sh b/git-submodule.sh
index 7f11158..a1bc8d5 100755
--- a/git-submodule.sh
+++ b/git-submodule.sh
@@ -625,6 +625,89 @@ cmd_update_clone()
 	cmd_update_recursive
 }
 
+cmd_update_fetch()
+{
+	subsha1=$(clear_local_git_env; cd "$sm_path" &&
+		git rev-parse --verify HEAD) ||
+	die "$(eval_gettext "Unable to find current revision in submodule path '\$displaypath'")"
+
+	if test -n "$remote"
+	then
+		if test -z "$nofetch"
+		then
+			# Fetch remote before determining tracking $sha1
+			(clear_local_git_env; cd "$sm_path" && git-fetch) ||
+			die "$(eval_gettext "Unable to fetch in submodule path '\$sm_path'")"
+		fi
+		remote_name=$(clear_local_git_env; cd "$sm_path" && get_default_remote)
+		sha1=$(clear_local_git_env; cd "$sm_path" &&
+			git rev-parse --verify "${remote_name}/${branch}") ||
+		die "$(eval_gettext "Unable to find current ${remote_name}/${branch} revision in submodule path '\$sm_path'")"
+	fi
+
+	if test "$subsha1" != "$sha1" || test -n "$force"
+	then
+		subforce=$force
+		# If we don't already have a -f flag and the submodule has never been checked out
+		if test -z "$subsha1" && test -z "$force"
+		then
+			subforce="-f"
+		fi
+
+		if test -z "$nofetch"
+		then
+			# Run fetch only if $sha1 isn't present or it
+			# is not reachable from a ref.
+			(clear_local_git_env; cd "$sm_path" &&
+				( (rev=$(git rev-list -n 1 $sha1 --not --all 2>/dev/null) &&
+				 test -z "$rev") || git-fetch)) ||
+			die "$(eval_gettext "Unable to fetch in submodule path '\$displaypath'")"
+		fi
+
+		must_die_on_failure=
+		case "$update_module" in
+		checkout)
+			command="git checkout $subforce -q"
+			die_msg="$(eval_gettext "Unable to checkout '\$sha1' in submodule path '\$displaypath'")"
+			say_msg="$(eval_gettext "Submodule path '\$displaypath': checked out '\$sha1'")"
+			;;
+		rebase)
+			command="git rebase"
+			die_msg="$(eval_gettext "Unable to rebase '\$sha1' in submodule path '\$displaypath'")"
+			say_msg="$(eval_gettext "Submodule path '\$displaypath': rebased into '\$sha1'")"
+			must_die_on_failure=yes
+			;;
+		merge)
+			command="git merge"
+			die_msg="$(eval_gettext "Unable to merge '\$sha1' in submodule path '\$displaypath'")"
+			say_msg="$(eval_gettext "Submodule path '\$displaypath': merged in '\$sha1'")"
+			must_die_on_failure=yes
+			;;
+		!*)
+			command="${update_module#!}"
+			die_msg="$(eval_gettext "Execution of '\$command \$sha1' failed in submodule  path '\$prefix\$sm_path'")"
+			say_msg="$(eval_gettext "Submodule path '\$prefix\$sm_path': '\$command \$sha1'")"
+			must_die_on_failure=yes
+			;;
+		*)
+			die "$(eval_gettext "Invalid update mode '$update_module' for submodule '$name'")"
+		esac
+
+		if (clear_local_git_env; cd "$sm_path" && $command "$sha1")
+		then
+			say "$say_msg"
+		elif test -n "$must_die_on_failure"
+		then
+			die_with_status 2 "$die_msg"
+		else
+			err="${err};$die_msg"
+			return
+		fi
+	fi
+
+	cmd_update_recursive
+}
+
 #
 # Update each submodule path to correct revision, using clone and checkout as needed
 #
@@ -743,88 +826,9 @@ Maybe you want to use 'update --init'?")"
 		if ! test -d "$sm_path"/.git && ! test -f "$sm_path"/.git
 		then
 			cmd_update_clone
-			continue
 		else
-			subsha1=$(clear_local_git_env; cd "$sm_path" &&
-				git rev-parse --verify HEAD) ||
-			die "$(eval_gettext "Unable to find current revision in submodule path '\$displaypath'")"
+			cmd_update_fetch
 		fi
-
-		if test -n "$remote"
-		then
-			if test -z "$nofetch"
-			then
-				# Fetch remote before determining tracking $sha1
-				(clear_local_git_env; cd "$sm_path" && git-fetch) ||
-				die "$(eval_gettext "Unable to fetch in submodule path '\$sm_path'")"
-			fi
-			remote_name=$(clear_local_git_env; cd "$sm_path" && get_default_remote)
-			sha1=$(clear_local_git_env; cd "$sm_path" &&
-				git rev-parse --verify "${remote_name}/${branch}") ||
-			die "$(eval_gettext "Unable to find current ${remote_name}/${branch} revision in submodule path '\$sm_path'")"
-		fi
-
-		if test "$subsha1" != "$sha1" || test -n "$force"
-		then
-			subforce=$force
-			# If we don't already have a -f flag and the submodule has never been checked out
-			if test -z "$subsha1" && test -z "$force"
-			then
-				subforce="-f"
-			fi
-
-			if test -z "$nofetch"
-			then
-				# Run fetch only if $sha1 isn't present or it
-				# is not reachable from a ref.
-				(clear_local_git_env; cd "$sm_path" &&
-					( (rev=$(git rev-list -n 1 $sha1 --not --all 2>/dev/null) &&
-					 test -z "$rev") || git-fetch)) ||
-				die "$(eval_gettext "Unable to fetch in submodule path '\$displaypath'")"
-			fi
-
-			must_die_on_failure=
-			case "$update_module" in
-			checkout)
-				command="git checkout $subforce -q"
-				die_msg="$(eval_gettext "Unable to checkout '\$sha1' in submodule path '\$displaypath'")"
-				say_msg="$(eval_gettext "Submodule path '\$displaypath': checked out '\$sha1'")"
-				;;
-			rebase)
-				command="git rebase"
-				die_msg="$(eval_gettext "Unable to rebase '\$sha1' in submodule path '\$displaypath'")"
-				say_msg="$(eval_gettext "Submodule path '\$displaypath': rebased into '\$sha1'")"
-				must_die_on_failure=yes
-				;;
-			merge)
-				command="git merge"
-				die_msg="$(eval_gettext "Unable to merge '\$sha1' in submodule path '\$displaypath'")"
-				say_msg="$(eval_gettext "Submodule path '\$displaypath': merged in '\$sha1'")"
-				must_die_on_failure=yes
-				;;
-			!*)
-				command="${update_module#!}"
-				die_msg="$(eval_gettext "Execution of '\$command \$sha1' failed in submodule  path '\$prefix\$sm_path'")"
-				say_msg="$(eval_gettext "Submodule path '\$prefix\$sm_path': '\$command \$sha1'")"
-				must_die_on_failure=yes
-				;;
-			*)
-				die "$(eval_gettext "Invalid update mode '$update_module' for submodule '$name'")"
-			esac
-
-			if (clear_local_git_env; cd "$sm_path" && $command "$sha1")
-			then
-				say "$say_msg"
-			elif test -n "$must_die_on_failure"
-			then
-				die_with_status 2 "$die_msg"
-			else
-				err="${err};$die_msg"
-				continue
-			fi
-		fi
-
-		cmd_update_recursive
 	done
 
 	if test -n "$err"
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCHv4 14/14] Rewrite submodule update in C
  2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
                   ` (12 preceding siblings ...)
  2015-09-23  1:45 ` [PATCHv4 13/14] git submodule update: cmd_update_fetch Stefan Beller
@ 2015-09-23  1:45 ` Stefan Beller
  13 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-23  1:45 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

This will make parallelisation easier in a followup patch. This is just
a translation from shell to C, hopefully not introducing any bugs.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 builtin/submodule--helper.c | 251 ++++++++++++++++++++++++++++++++++++++++++++
 git-submodule.sh            | 135 +-----------------------
 2 files changed, 254 insertions(+), 132 deletions(-)

diff --git a/builtin/submodule--helper.c b/builtin/submodule--helper.c
index f4c3eff..b79117a 100644
--- a/builtin/submodule--helper.c
+++ b/builtin/submodule--helper.c
@@ -255,6 +255,256 @@ static int module_clone(int argc, const char **argv, const char *prefix)
 	return 0;
 }
 
+struct module_update_data {
+	struct module_list list;
+	int count;
+
+	struct pathspec pathspec;
+
+	int no_fetch;
+	int force;
+	int update;
+	int quiet;
+	int recursive;
+	int remote;
+	char *prefix;
+	char *reference;
+	char *depth;
+
+	struct argv_array args;
+
+	int result;
+};
+
+static void module_update_data_init(struct module_update_data *mud)
+{
+	mud->list.entries = NULL;
+	mud->list.alloc = 0;
+	mud->list.nr = 0;
+
+	memset(&mud->pathspec, 0, sizeof(mud->pathspec));
+
+	mud->count = 0;
+	mud->no_fetch = 0;
+	mud->force = 0;
+	mud->update = 0;
+	mud->quiet = 0;
+	mud->remote = 0;
+	mud->recursive = 0;
+	mud->result = 0;
+
+	mud->prefix = NULL;
+	mud->reference = NULL;
+	mud->depth = NULL;
+
+	argv_array_init(&mud->args);
+}
+
+static int update_next_task(void *data,
+		     struct child_process *cp,
+		     struct strbuf *err)
+{
+	int i;
+	struct module_update_data *mud = data;
+	struct strbuf sb = STRBUF_INIT;
+	const char *displaypath;
+
+	for (; mud->count < mud->list.nr; mud->count++) {
+		const char *update_module;
+		const char *sm_gitdir;
+		const struct submodule *sub;
+		const struct cache_entry *ce = mud->list.entries[mud->count];
+
+		displaypath = relative_path(ce->name, mud->prefix, &sb);
+		strbuf_reset(&sb);
+
+		if (ce_stage(ce)) {
+			strbuf_addf(err, "Skipping unmerged submodule %s",
+				    displaypath);
+			continue;
+		}
+
+		sub = submodule_from_path(null_sha1, ce->name);
+		if (!sub) {
+			mud->result = 1;
+			return 0;
+		}
+
+		switch (mud->update) {
+		case 'r':
+			update_module = "rebase";
+			break;
+		case 'c':
+			update_module = "checkout";
+			break;
+		case 'm':
+			update_module = "merge";
+			break;
+		case 0:
+			/* not specified by command line */
+			if (sub->update)
+				update_module = sub->update;
+			else
+				update_module = "checkout";
+			break;
+		default:
+			die("BUG: update mode not covered");
+		}
+
+		if (!strcmp(update_module, "none")) {
+			strbuf_addf(err, "Skipping submodule '%s'", displaypath);
+			continue;
+		}
+
+		if (!sub->url) {
+			/*
+			 * Only mention uninitialized submodules when its
+			 * path have been specified
+			 */
+			if (!mud->pathspec.nr)
+				continue;
+
+			strbuf_addf(err,
+				    _("Submodule path '%s' not initialized \n"
+				    "Maybe you want to use 'update --init'?"),
+				    displaypath);
+			continue;
+		}
+
+		strbuf_addf(&sb, "%s/.git", ce->name);
+		sm_gitdir = strbuf_detach(&sb, NULL);
+
+		child_process_init(cp);
+		for (i = 0; local_repo_env[i]; i++)
+			argv_array_pushf(&cp->env_array, "%s", local_repo_env[i]);
+
+		argv_array_pushf(&cp->env_array, "displaypath=%s", displaypath);
+		argv_array_pushf(&cp->env_array, "sm_path=%s", sub->path);
+		argv_array_pushf(&cp->env_array, "name=%s", sub->name);
+		argv_array_pushf(&cp->env_array, "url=%s", sub->url);
+		argv_array_pushf(&cp->env_array, "sha1=%s", sha1_to_hex(ce->sha1));
+		argv_array_pushf(&cp->env_array, "update_module=%s", update_module);
+
+		cp->git_cmd = 1;
+		cp->no_stdin = 1;
+		cp->stdout_to_stderr = 1;
+		cp->err = -1;
+		argv_array_init(&cp->args);
+		argv_array_push(&cp->args, "submodule");
+		if (!file_exists(sm_gitdir))
+			argv_array_push(&cp->args, "update_clone");
+		else
+			argv_array_push(&cp->args, "update_fetch");
+
+		argv_array_pushf(&cp->args, "%s", ce->name);
+		mud->count++;
+		return 1;
+	}
+	return 0;
+}
+
+void update_subcommand_failure(void *data,
+			       struct child_process *cp,
+			       struct strbuf *err)
+{
+	struct module_update_data *mud = data;
+	strbuf_addf(err, _("Could not start child process"));
+	mud->result = 1;
+}
+
+void update_child_return(void *data,
+			 struct child_process *cp,
+			 int result)
+{
+	struct module_update_data *mud = data;
+	mud->result = 1;
+}
+
+static int module_update(int argc, const char **argv, const char *prefix)
+{
+	int init;
+	struct module_update_data mud;
+
+	struct option module_list_options[] = {
+		OPT_STRING(0, "prefix", &prefix,
+			   N_("path"),
+			   N_("alternative anchor for relative paths")),
+		OPT_BOOL('i', "init", &init,
+			N_("Initialize the submodule if not yet done")),
+		OPT_BOOL(0, "remote", &mud.remote,
+			N_("Update the submodule to the remote branch instead "
+			   "of the superprojects specification")),
+		OPT_BOOL('N', "no-fetch", &mud.no_fetch,
+			N_("Don’t fetch new objects from the remote site.")),
+		OPT_BOOL('f', "force", &mud.force,
+			N_("Ignore local changes in submodules")),
+		OPT_CMDMODE('r', "rebase", &mud.update,
+			N_("Rebase local changes in submodules"), 'r'),
+		OPT_CMDMODE('m', "merge", &mud.update,
+			N_("Merge local changes in submodules"), 'm'),
+		OPT_CMDMODE(0, "checkout", &mud.update,
+			N_("Checkout to a detached HEAD in submodules"), 'c'),
+		OPT_BOOL(0, "recursive", &mud.recursive,
+			N_("Update nested submodules")),
+		OPT_STRING(0, "reference", &mud.reference, "<repository>",
+			N_("Use the local reference repository "
+			   "instead of a full clone")),
+		OPT_STRING(0, "depth", &mud.depth, "<depth>",
+			N_("Create a shallow clone truncated to the "
+			   "specified number of revisions")),
+		OPT__QUIET(&mud.quiet, N_("be quiet")),
+		OPT_END()
+	};
+
+	const char *const git_submodule_helper_usage[] = {
+		N_("git submodule--helper list [--prefix=<path>] [<path>...]"),
+		NULL
+	};
+
+	module_update_data_init(&mud);
+	gitmodules_config();
+
+	argc = parse_options(argc, argv, prefix, module_list_options,
+			     git_submodule_helper_usage, 0);
+
+	if (mud.force)
+		argv_array_push(&mud.args, "force=1");
+	if (mud.quiet)
+		argv_array_push(&mud.args, "GIT_QUIET=1");
+	if (mud.recursive)
+		argv_array_push(&mud.args, "recursive=1");
+	if (mud.prefix)
+		argv_array_pushf(&mud.args, "prefix=%s", mud.prefix);
+	if (mud.reference)
+		argv_array_pushf(&mud.args, "reference=%s", mud.reference);
+	if (mud.depth)
+		argv_array_pushf(&mud.args, "depth=%s", mud.depth);
+
+	if (module_list_compute(argc, argv, prefix, &mud.pathspec, &mud.list) < 0)
+		return 1;
+
+	if (init) {
+		const char **argv_init = xmalloc((2 + mud.list.nr) * sizeof(char*));
+		int argc = 0, i, code;
+		argv_init[argc++] = "submodule";
+		argv_init[argc++] = "init";
+
+		for (i = 0; i < mud.list.nr; i++) {
+			const struct cache_entry *ce = mud.list.entries[i];
+			argv_init[argc++] = ce->name;
+		}
+		code = run_command_v_opt(argv_init, RUN_GIT_CMD);
+			if (code)
+				return code;
+	}
+
+	run_processes_parallel(1, &mud,
+			       update_next_task,
+			       update_subcommand_failure,
+			       update_child_return);
+	return 0;
+}
+
 struct cmd_struct {
 	const char *cmd;
 	int (*fn)(int, const char **, const char *);
@@ -264,6 +514,7 @@ static struct cmd_struct commands[] = {
 	{"list", module_list},
 	{"name", module_name},
 	{"clone", module_clone},
+	{"update", module_update}
 };
 
 int cmd_submodule__helper(int argc, const char **argv, const char *prefix)
diff --git a/git-submodule.sh b/git-submodule.sh
index a1bc8d5..63e9b3b 100755
--- a/git-submodule.sh
+++ b/git-submodule.sh
@@ -640,6 +640,7 @@ cmd_update_fetch()
 			die "$(eval_gettext "Unable to fetch in submodule path '\$sm_path'")"
 		fi
 		remote_name=$(clear_local_git_env; cd "$sm_path" && get_default_remote)
+		branch=$(get_submodule_config "$name" branch master)
 		sha1=$(clear_local_git_env; cd "$sm_path" &&
 			git rev-parse --verify "${remote_name}/${branch}") ||
 		die "$(eval_gettext "Unable to find current ${remote_name}/${branch} revision in submodule path '\$sm_path'")"
@@ -715,137 +716,7 @@ cmd_update_fetch()
 #
 cmd_update()
 {
-	# parse $args after "submodule ... update".
-	while test $# -ne 0
-	do
-		case "$1" in
-		-q|--quiet)
-			GIT_QUIET=1
-			;;
-		-i|--init)
-			init=1
-			;;
-		--remote)
-			remote=1
-			;;
-		-N|--no-fetch)
-			nofetch=1
-			;;
-		-f|--force)
-			force=$1
-			;;
-		-r|--rebase)
-			update="rebase"
-			;;
-		--reference)
-			case "$2" in '') usage ;; esac
-			reference="--reference=$2"
-			shift
-			;;
-		--reference=*)
-			reference="$1"
-			;;
-		-m|--merge)
-			update="merge"
-			;;
-		--recursive)
-			recursive=1
-			;;
-		--checkout)
-			update="checkout"
-			;;
-		--depth)
-			case "$2" in '') usage ;; esac
-			depth="--depth=$2"
-			shift
-			;;
-		--depth=*)
-			depth=$1
-			;;
-		--)
-			shift
-			break
-			;;
-		-*)
-			usage
-			;;
-		*)
-			break
-			;;
-		esac
-		shift
-	done
-
-	if test -n "$init"
-	then
-		cmd_init "--" "$@" || return
-	fi
-
-	git submodule--helper list --prefix "$wt_prefix" "$@" | {
-	err=
-	while read mode sha1 stage sm_path
-	do
-		die_if_unmatched "$mode"
-		if test "$stage" = U
-		then
-			echo >&2 "Skipping unmerged submodule $prefix$sm_path"
-			continue
-		fi
-		name=$(git submodule--helper name "$sm_path") || exit
-		url=$(git config submodule."$name".url)
-		branch=$(get_submodule_config "$name" branch master)
-		if ! test -z "$update"
-		then
-			update_module=$update
-		else
-			update_module=$(git config submodule."$name".update)
-			if test -z "$update_module"
-			then
-				update_module="checkout"
-			fi
-		fi
-
-		displaypath=$(relative_path "$prefix$sm_path")
-
-		if test "$update_module" = "none"
-		then
-			echo "Skipping submodule '$displaypath'"
-			continue
-		fi
-
-		if test -z "$url"
-		then
-			# Only mention uninitialized submodules when its
-			# path have been specified
-			test "$#" != "0" &&
-			say "$(eval_gettext "Submodule path '\$displaypath' not initialized
-Maybe you want to use 'update --init'?")"
-			continue
-		fi
-
-		if ! test -d "$sm_path"/.git && ! test -f "$sm_path"/.git
-		then
-			cmd_update_clone
-		else
-			cmd_update_fetch
-		fi
-	done
-
-	if test -n "$err"
-	then
-		OIFS=$IFS
-		IFS=';'
-		for e in $err
-		do
-			if test -n "$e"
-			then
-				echo >&2 "$e"
-			fi
-		done
-		IFS=$OIFS
-		exit 1
-	fi
-	}
+	git submodule--helper update ${prefix:+--prefix "$prefix"} "$@"
 }
 
 set_name_rev () {
@@ -1243,7 +1114,7 @@ cmd_sync()
 while test $# != 0 && test -z "$command"
 do
 	case "$1" in
-	add | foreach | init | deinit | update | status | summary | sync)
+	add | foreach | init | deinit | update | update_fetch | update_clone | status | summary | sync)
 		command=$1
 		;;
 	-q|--quiet)
-- 
2.5.0.272.ga84127c.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23  1:45 ` [PATCHv4 06/14] run-command: add an asynchronous parallel child processor Stefan Beller
@ 2015-09-23  6:29   ` Junio C Hamano
  2015-09-23 17:53     ` Stefan Beller
  2015-09-23  6:47   ` Junio C Hamano
  1 sibling, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23  6:29 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Stefan Beller <sbeller@google.com> writes:

>  run-command.c          | 264 +++++++++++++++++++++++++++++++++++++++++++++++++
>  run-command.h          |  36 +++++++
>  t/t0061-run-command.sh |  20 ++++
>  test-run-command.c     |  24 +++++
>  4 files changed, 344 insertions(+)

I think we are almost there, but there were still a few more "huh?"
in this patch.

> +/* returns 1 if a process was started, 0 otherwise */

This claim is dubious.

The last four lines of this function marks the i-th slot as in_use,
and increments nr_processes, so that exit codepath clearly is about
"a process was started" and should be returning 1, but the code
returns 0, which looks incorrect.

> +static int pp_start_one(struct parallel_processes *pp)
> +{
> +	int i;
> +
> +	for (i = 0; i < pp->max_processes; i++)
> +		if (!pp->children[i].in_use)
> +			break;
> +	if (i == pp->max_processes)
> +		die("BUG: bookkeeping is hard");
> +
> +	if (!pp->get_next_task(pp->data,
> +			       &pp->children[i].process,
> +			       &pp->children[i].err))
> +		return 1;

And this one, when get_next_task() says "nothing more to do", is
clearly "we returned without starting anything", so according to the
comment it should be returning 0, but the code returns 1, which
looks incorrect.

> +	if (start_command(&pp->children[i].process))
> +		pp->start_failure(pp->data,
> +				  &pp->children[i].process,
> +				  &pp->children[i].err);

What should happen if start_failure returns without dying?
Shouldn't this function return something, without doing the
remainder of it?  i.e.

	if (start_command(...)) {
		pp->start_failur(...);
                return SOMETHING;
	}

According to the comment at the beginning, that SOMETHING ought to
be 0, because we did not spawn anything, i.e. we did not increment
nr_processes.

But don't make that change yet; I do not think it is a great
interface to say "did we or did we not add a new process?" anyway
(see below).

> +	set_nonblocking(pp->children[i].process.err);
> +
> +	pp->nr_processes++;
> +	pp->children[i].in_use = 1;
> +	pp->pfd[i].fd = pp->children[i].process.err;
> +	return 0;

And this is "we spawned" that ought to have returned 1.

Perhaps the comment at the beginning is wrong, as the code
consistently does the opposite of what the comment says.

But it does not really matter, as I do not think this should be
returning "did we or did we not start a new process?" (see below).

> +}

> +int run_processes_parallel(int n, void *data,
> +			   get_next_task_fn get_next_task,
> +			   start_failure_fn start_failure,
> +			   return_value_fn return_value)
> +{
> +	struct parallel_processes pp;
> +	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
> +
> +	while (1) {
> +		while (pp.nr_processes < pp.max_processes &&
> +		       !pp_start_one(&pp))
> +			; /* nothing */
> +		if (!pp.nr_processes)
> +			break;

This inner loop is why I think "did we or did we not spawn a new
process?" is not a great interface.

The reason why it is not a great interface is because there are two
possible reasons why pp_start_one() does not spawn a new process,
and this caller wants to behave differently depending on why it did
not spawn a new process.  They are:

 * get_next_task() truly ran out of things to do.

 * get_next_task() gave us a task, but it did not start, and
   start_failure was set not to die (e.g. the function can be used
   to tell the next_task machinery that it needs to return a
   replacement task for the one that failed to run.  That way, upon
   next call to get_next_task, a replacement task can run instead of
   the old one that failed).

For the former, we want to stop looping, for the latter, we
definitely do want to keep looping, as we want to make another call
to get_next_task() to grab the replacement task for the one that
just failed.

So I think it makes more sense to define the meaning of the return
value from pp_start_one() differently from the way this patch
defines.  "Return 0 when we truly ran out of things to do, otherwise
return non-zero", for example, would make more sense.  The return
value does not tell you if the call resulted in one more process,
but that is not any loss, as you can look at pp.nr_processes
yourself if you really cared.

With that, the above caller could be updated, with optional gradual
ramp_up, like so:

	#define RAMP_UP_LIMIT 2

	while (1) {
		int ramp_up;
		int no_more_task;

		for (no_more_task = 0, ramp_up = RAMP_UP_LIMIT;
                     !no_more_task && ramp_up && pp.nr_processes < pp.max_processes;
		     ramp_up--)
			if (!pp_start_one(&pp))
				no_more_task = 1;

		if (!pp.nr_processes && no_more_task)
			break;

If you prefer to swamp the system with a thundering herd at the
beginning, you can define RAMP_UP_LIMIT to really a high value
instead, e.g. "#define RAMPUP_LIMIT pp.max_processes".  I however
would not recommend it because doing so would hurt the perceived
latency at the beginning.

After the system goes into a steady state, how you set RAMP_UP_LIMIT
would not make that much difference, as your slots should be almost
always full and you will be replenishing an open slot with a single
task as each running task finishes, and you would not be running
more than one pp_start_one() at a time anyway.

> +		pp_buffer_stderr(&pp);
> +		pp_output(&pp);
> +		pp_collect_finished(&pp);
> +	}
> +
> +	pp_cleanup(&pp);
> +
> +	return 0;
> +}

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23  1:45 ` [PATCHv4 06/14] run-command: add an asynchronous parallel child processor Stefan Beller
  2015-09-23  6:29   ` Junio C Hamano
@ 2015-09-23  6:47   ` Junio C Hamano
  2015-09-23 14:59     ` Junio C Hamano
  1 sibling, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23  6:47 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Stefan Beller <sbeller@google.com> writes:

> +static void pp_buffer_stderr(struct parallel_processes *pp)
> +{
> +	int i;
> +
> +	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
> +		if (errno == EINTR)
> +			continue;
> +		pp_cleanup(pp);
> +		die_errno("poll");
> +	}
> +
> +	/* Buffer output from all pipes. */
> +	for (i = 0; i < pp->max_processes; i++) {
> +		if (pp->children[i].in_use &&
> +		    pp->pfd[i].revents & POLLIN)
> +			if (strbuf_read_once(&pp->children[i].err,
> +					     pp->children[i].process.err, 0) < 0)
> +				if (errno != EAGAIN)
> +					die_errno("read");
> +	}
> +}

I think it is a good thing that the caller is passing the whole pp
to this function.  One thing you may want to consider is to adjust
the poll(2) timeout longer when the process slots are full.

There is nothing you can gain by returning early due to timeout
without doing anything from this function when you know you cannot
start a new process (here, I am assuming that your poll(2) would be
unblocked for a disconnect when one of the processes exits, letting
you return and letting the caller call collect_finished(), which in
turn would allow us to make some progress).

On the other hand, during the early ramp-up period, you may want to
use a shorter poll(2) timeout to give the caller a chance to spawn
more processes sooner.  But that falls into performance tuning that
can and should be left to a later follow-up patch after we get the
basic machinery right.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23  6:47   ` Junio C Hamano
@ 2015-09-23 14:59     ` Junio C Hamano
  2015-09-23 17:54       ` Junio C Hamano
  0 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 14:59 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Junio C Hamano <gitster@pobox.com> writes:

> Stefan Beller <sbeller@google.com> writes:
>
>> +static void pp_buffer_stderr(struct parallel_processes *pp)
>> +{
>> +	int i;
>> +
>> +	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
>> +		if (errno == EINTR)
>> +			continue;
>> +		pp_cleanup(pp);
>> +		die_errno("poll");
>> +	}
>> +
>> +	/* Buffer output from all pipes. */
>> +	for (i = 0; i < pp->max_processes; i++) {
>> +		if (pp->children[i].in_use &&
>> +		    pp->pfd[i].revents & POLLIN)
>> +			if (strbuf_read_once(&pp->children[i].err,
>> +					     pp->children[i].process.err, 0) < 0)
>> +				if (errno != EAGAIN)
>> +					die_errno("read");
>> +	}
>> +}
>
> I think it is a good thing that the caller is passing the whole pp
> to this function.  One thing you may want to consider is to adjust
> the poll(2) timeout longer when the process slots are full.
> ...
> But that falls into performance tuning that
> can and should be left to a later follow-up patch after we get the
> basic machinery right.

Just to make sure there is no misunderstanding, just like I prefer
"start one" over "start as many as possible" in order to give
scheduling decision to the calling loop, I would expect that such a
follow-up performance tuning would be done by adding another field
to pp tht lets the caller of this function, the overall scheduling
loop, to tweak the timeout used, without this function making the
decision locallly.

Thanks.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23  6:29   ` Junio C Hamano
@ 2015-09-23 17:53     ` Stefan Beller
  2015-09-23 18:04       ` Junio C Hamano
  0 siblings, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-23 17:53 UTC (permalink / raw)
  To: Junio C Hamano
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

On Tue, Sep 22, 2015 at 11:29 PM, Junio C Hamano <gitster@pobox.com> wrote:

>
> And this one, when get_next_task() says "nothing more to do", is
> clearly "we returned without starting anything", so according to the
> comment it should be returning 0, but the code returns 1, which
> looks incorrect.
>
>> +     if (start_command(&pp->children[i].process))
>> +             pp->start_failure(pp->data,
>> +                               &pp->children[i].process,
>> +                               &pp->children[i].err);
>
> What should happen if start_failure returns without dying?
> Shouldn't this function return something, without doing the
> remainder of it?  i.e.
>
>         if (start_command(...)) {
>                 pp->start_failur(...);
>                 return SOMETHING;
>         }

Right, I forgot about that code path as I was blinded by the obviousness
("If you cannot start a process, of course we'll die").

So for this SOMETHING we need to decide if it should signal
that an immediate retry can be done. But then we could perform the
immediate retry ourselves:

startnewtask:
    if (!pp->get_next_task(pp->data,
                   &pp->children[i].process,
                   &pp->children[i].err))
        return 0;

    if (start_command(&pp->children[i].process)) {
        pp->start_failure(pp->data,
                  &pp->children[i].process,
                  &pp->children[i].err);
        goto startnewtask;
    }

But this could result in an endless loop.
Even if we would decide to return to the caller run_processes_parallel
and let them decide to try again, this version of the patch may produce an
infinite loop there.

The other alternative would be to make SOMETHING signal to not
immediately try again. ("We failed to start a child process, give it some time
by doing the poll/output and try again")

This however could not finish all workloads reliably as we may fail to start
the first child, such that there are 0 children processes running and the
control loop in run_processes_parallel shuts down the whole parallel processor.

So for now I'd lean on having the SOMETHING be the same boolean as
a successful start (failure -1, successful start 1, no more pending work 0)
and the difference between -1 and 1 can be sorted out in a later patch, which
introduces workloads with failing children.

>> +
>> +     while (1) {
>> +             while (pp.nr_processes < pp.max_processes &&
>> +                    !pp_start_one(&pp))
>> +                     ; /* nothing */
>> +             if (!pp.nr_processes)
>> +                     break;
>
> This inner loop is why I think "did we or did we not spawn a new
> process?" is not a great interface.

Right, we actually need to return whether we have nothing more to do
("Don't even try to call me again") or if we did something useful and expect
to do more useful things in the next call. (Either starting anew command or
finding out it failed).

This would be indicated by the -1/1/0 return signals.

>
> The reason why it is not a great interface is because there are two
> possible reasons why pp_start_one() does not spawn a new process,
> and this caller wants to behave differently depending on why it did
> not spawn a new process.  They are:
>
>  * get_next_task() truly ran out of things to do.
>
>  * get_next_task() gave us a task, but it did not start, and
>    start_failure was set not to die (e.g. the function can be used
>    to tell the next_task machinery that it needs to return a
>    replacement task for the one that failed to run.  That way, upon
>    next call to get_next_task, a replacement task can run instead of
>    the old one that failed).
>
> For the former, we want to stop looping, for the latter, we
> definitely do want to keep looping, as we want to make another call
> to get_next_task() to grab the replacement task for the one that
> just failed.
>
> So I think it makes more sense to define the meaning of the return
> value from pp_start_one() differently from the way this patch
> defines.  "Return 0 when we truly ran out of things to do, otherwise
> return non-zero", for example, would make more sense.

ok, we have the same opinion. I just documented poorly.

>  The return
> value does not tell you if the call resulted in one more process,
> but that is not any loss, as you can look at pp.nr_processes
> yourself if you really cared.
>
> With that, the above caller could be updated, with optional gradual
> ramp_up, like so:
>
>         #define RAMP_UP_LIMIT 2
>
>         while (1) {
>                 int ramp_up;
>                 int no_more_task;
>
>                 for (no_more_task = 0, ramp_up = RAMP_UP_LIMIT;
>                      !no_more_task && ramp_up && pp.nr_processes < pp.max_processes;
>                      ramp_up--)
>                         if (!pp_start_one(&pp))
>                                 no_more_task = 1;

I would not have the no_more_task variable, but just reuse
ramp_up and set it to zero in case of !pp_start_one(&pp).

I am not sure if the ramp up machinery is really needed.
I modified the test-run-command test function to start up to 400 processes.
(Most people will use less than 400 processes in the next 5 years), and run
just as in t0061:

    ./test-run-command run-command-parallel-400 sh -c "printf
\"%s\n%s\n\" Hello World"

The output felt immediate (not slowed down or anything). The numbers seem to
support that

    real 0m0.110s
    user 0m0.045s
    sys 0m0.366s

Any delay below 0.1 second cannot really be perceived by a human. You
can sure tell
a difference of 0.1 second in say 2 acoustic signals or light flashes,
but you cannot
tell that the output "was slow". So IMHO the ramp up machinery doesn't
have a high
priority for now.


>
>                 if (!pp.nr_processes && no_more_task)
>                         break;
>
> If you prefer to swamp the system with a thundering herd at the
> beginning, you can define RAMP_UP_LIMIT to really a high value
> instead, e.g. "#define RAMPUP_LIMIT pp.max_processes".  I however
> would not recommend it because doing so would hurt the perceived
> latency at the beginning.
>
> After the system goes into a steady state, how you set RAMP_UP_LIMIT
> would not make that much difference, as your slots should be almost
> always full and you will be replenishing an open slot with a single
> task as each running task finishes, and you would not be running
> more than one pp_start_one() at a time anyway.

Yeah there we could have a simple

    if (pp->nr_processes == pp->max_processes)
        poll_timeout = 5 seconds
    else
        poll_timeout = 10..100 milliseconds

>
>> +             pp_buffer_stderr(&pp);
>> +             pp_output(&pp);
>> +             pp_collect_finished(&pp);
>> +     }
>> +
>> +     pp_cleanup(&pp);
>> +
>> +     return 0;
>> +}

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23 14:59     ` Junio C Hamano
@ 2015-09-23 17:54       ` Junio C Hamano
  2015-09-23 23:41         ` [PATCHv5] Another squash on " Stefan Beller
  0 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 17:54 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Junio C Hamano <gitster@pobox.com> writes:

> Just to make sure there is no misunderstanding, just like I prefer
> "start one" over "start as many as possible" in order to give
> scheduling decision to the calling loop, I would expect that...

To sum up, what I anticipate would happen over time on top of 06/14
is something like this:

 * pp_start_one() stuffed unborn child to the list of children when
   start_command() failed and start_failure() did not die(); return
   to the caller without corrupting children[] list in this case.

 * update the semantics of the value returned from pp_start_one()
   and adjust the scheduling loop for it.

 * allow slow-start of the whole process, so that we do not spawn
   tons of processes before starting to read from any of them, to
   give a better first-byte latency.  This is parameterized and
   in this squash, it is set not to do a slow-start.
    
 * make poll(2) timeout used in pp_buffer_stderr() configurable by
   the scheduing loop.  Raise the timeout when we are already running
   as many children as we would at the same time, and lower it when
   we are still ramping up.

This squash itself is a mixed bag of things.  The first point is to
fix a real issue, while the rest are merely envisioning possible
future.  The "envisioning possible future" parts may be illustrative
when deciding what design we want in the basic structure.

An imaginary alternative version would have the top-level loop that
is a mere "these are the things we do" bullet-list that runs "spawn
processes", "slurp in their output with some timeout", "output if
foreground process is ready", and "cull finished children" in order,
which was your original.  I would imagine that we would teach "spawn
processes" part to slow-start, "slurp" part to adjust timeout
depending on the fullness of children[] and if the slow-start logic
is still ramping up, etc., in such a code structure by keeping
fields in *pp that corresponds to 'cnt' and 'no_more_task' variables
and have these four steps in the bullet-list communiate and
coordinate among themselves using those fields.

Compared to code that is structured that way, I think the top-level
loop that owns 'cnt' and 'child_timeout' variables to make the
scheduling decisions on its own, and drives "dumb" helper functions
to drive the whole system, which is what this squash attempts to
create, makes the overall logic and structure much clearer to see.

Thanks.

diff --git a/run-command.c b/run-command.c
index 494e1f8..b6d8b39 100644
--- a/run-command.c
+++ b/run-command.c
@@ -977,7 +977,7 @@ static void set_nonblocking(int fd)
 			"output will be degraded");
 }
 
-/* returns 1 if a process was started, 0 otherwise */
+/* return 0 if get_next_task() ran out of things to do, non-zero otherwise */
 static int pp_start_one(struct parallel_processes *pp)
 {
 	int i;
@@ -991,26 +991,28 @@ static int pp_start_one(struct parallel_processes *pp)
 	if (!pp->get_next_task(pp->data,
 			       &pp->children[i].process,
 			       &pp->children[i].err))
-		return 1;
+		return 0;
 
-	if (start_command(&pp->children[i].process))
+	if (start_command(&pp->children[i].process)) {
 		pp->start_failure(pp->data,
 				  &pp->children[i].process,
 				  &pp->children[i].err);
+		return -1;
+	}
 
 	set_nonblocking(pp->children[i].process.err);
 
 	pp->nr_processes++;
 	pp->children[i].in_use = 1;
 	pp->pfd[i].fd = pp->children[i].process.err;
-	return 0;
+	return 1;
 }
 
-static void pp_buffer_stderr(struct parallel_processes *pp)
+static void pp_buffer_stderr(struct parallel_processes *pp, int output_timeout)
 {
 	int i;
 
-	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
+	while ((i = poll(pp->pfd, pp->max_processes, output_timeout)) < 0) {
 		if (errno == EINTR)
 			continue;
 		pp_cleanup(pp);
@@ -1105,6 +1107,9 @@ static void pp_collect_finished(struct parallel_processes *pp)
 	}
 }
 
+
+#define SPAWN_CAP (pp.max_processes + 1) /* spawn as many as possible */
+
 int run_processes_parallel(int n, void *data,
 			   get_next_task_fn get_next_task,
 			   start_failure_fn start_failure,
@@ -1114,12 +1119,27 @@ int run_processes_parallel(int n, void *data,
 	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
 
 	while (1) {
-		while (pp.nr_processes < pp.max_processes &&
-		       !pp_start_one(&pp))
-			; /* nothing */
-		if (!pp.nr_processes)
+		int no_more_task, cnt, output_timeout;
+
+		for (cnt = SPAWN_CAP, no_more_task = 0;
+		     cnt && pp.nr_processes < pp.max_processes;
+		     cnt--) {
+			if (!pp_start_one(&pp)) {
+				no_more_task = 1;
+				break;
+			}
+		}
+
+		if (no_more_task && !pp.nr_processes)
 			break;
-		pp_buffer_stderr(&pp);
+		if (!cnt)
+			output_timeout = 50;
+		else if (pp.nr_processes < pp.max_processes)
+			output_timeout = 100;
+		else
+			output_timeout = 1000;
+		pp_buffer_stderr(&pp, output_timeout);
+
 		pp_output(&pp);
 		pp_collect_finished(&pp);
 	}

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23 17:53     ` Stefan Beller
@ 2015-09-23 18:04       ` Junio C Hamano
  2015-09-23 19:34         ` Junio C Hamano
  0 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 18:04 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Stefan Beller <sbeller@google.com> writes:

> I modified the test-run-command test function to start up to 400 processes.
> (Most people will use less than 400 processes in the next 5 years), and run
> just as in t0061:
>
>     ./test-run-command run-command-parallel-400 sh -c "printf
> \"%s\n%s\n\" Hello World"
>
> The output felt immediate (not slowed down or anything).

I doubt that such an experiment has any value.  You are not driving
printf in real life.

You are running "git fetch" that are is a lot more heavy-weight.
Because once each of them started fully they will be network bound,
it is likely that you would want to run more processes than you have
core.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23 18:04       ` Junio C Hamano
@ 2015-09-23 19:34         ` Junio C Hamano
  2015-09-23 19:39           ` Stefan Beller
  0 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 19:34 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Junio C Hamano <gitster@pobox.com> writes:

> You are running "git fetch" that are is a lot more heavy-weight.
> Because once each of them started fully they will be network bound,
> it is likely that you would want to run more processes than you have
> core.

I thought the conclusion would be obvious, but just in case the
readers need the flow of thought completed, from the above it
follows that on a N-core box (say 8-core) you may want to run the
fetch with -j16 (or more).  If we start everything at once, the time
before one process starts to produce first meaningful response (I am
not counting the "starting command for submodule-$i" message as
"meaningful") would take twice as long under such condition even if
you have infinite network bandwidth and talking to an infinitely
fast server.

I agree that a full slow-start ramping-up is not necessary in order
to hide the start-up lag.  All you need to do is to start just one
and make sure it becomes the foreground, and give it a time alone to
make enough progress to produce an early output without getting
slowed down by system activities caused by the other background
processes in the group that start all at the same time.  And once
the foreground process starts showing its output, you can unleash
the remainder of the herd to thunder and nobody would notice as they
run all in the background.  Again, I am assuming something like "git
fetch" that takes reasonably large amount of resource and time to
start-up and tear-down.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23 19:34         ` Junio C Hamano
@ 2015-09-23 19:39           ` Stefan Beller
  2015-09-23 19:47             ` Junio C Hamano
  0 siblings, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-23 19:39 UTC (permalink / raw)
  To: Junio C Hamano
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

On Wed, Sep 23, 2015 at 12:34 PM, Junio C Hamano <gitster@pobox.com> wrote:
> Junio C Hamano <gitster@pobox.com> writes:
>
>> You are running "git fetch" that are is a lot more heavy-weight.
>> Because once each of them started fully they will be network bound,
>> it is likely that you would want to run more processes than you have
>> core.
>
> I thought the conclusion would be obvious, but just in case the
> readers need the flow of thought completed, from the above it
> follows that on a N-core box (say 8-core) you may want to run the
> fetch with -j16 (or more).  If we start everything at once, the time
> before one process starts to produce first meaningful response (I am
> not counting the "starting command for submodule-$i" message as
> "meaningful") would take twice as long under such condition even if
> you have infinite network bandwidth and talking to an infinitely
> fast server.
>
> I agree that a full slow-start ramping-up is not necessary in order
> to hide the start-up lag.  All you need to do is to start just one
> and make sure it becomes the foreground, and give it a time alone to
> make enough progress to produce an early output without getting
> slowed down by system activities caused by the other background
> processes in the group that start all at the same time.  And once
> the foreground process starts showing its output, you can unleash
> the remainder of the herd to thunder and nobody would notice as they
> run all in the background.  Again, I am assuming something like "git
> fetch" that takes reasonably large amount of resource and time to
> start-up and tear-down.
>

I can confirm this now.

    git fetch --recurse-submodules=yes -j 400

in an submodule-ified Android tree takes very long to start putting out useful
information, but if I hardcode the SPAWN_CAP to 4 it looks pretty amazing
fast.

I'd guess we could have a reasonable default for SPAWN_CAP by

    #define SPAWN_CAP min(pp.max_processes + 1, online_cpus)

>

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 06/14] run-command: add an asynchronous parallel child processor
  2015-09-23 19:39           ` Stefan Beller
@ 2015-09-23 19:47             ` Junio C Hamano
  0 siblings, 0 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 19:47 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Stefan Beller <sbeller@google.com> writes:

> I can confirm this now.
>
>     git fetch --recurse-submodules=yes -j 400
>
> in an submodule-ified Android tree takes very long to start putting out useful
> information, but if I hardcode the SPAWN_CAP to 4 it looks pretty amazing
> fast.

Nice to hear that parallel fetching does work ;-)

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCHv4 12/14] git submodule update: cmd_update_clone
  2015-09-23  1:45 ` [PATCHv4 12/14] git submodule update: cmd_update_clone Stefan Beller
@ 2015-09-23 20:13   ` Junio C Hamano
  0 siblings, 0 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-23 20:13 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Stefan Beller <sbeller@google.com> writes:

> Split the cloning part out to its own function,
> this allow us in a later patch to convert cmd_update in C.
>
> Signed-off-by: Stefan Beller <sbeller@google.com>
> ---

It appears that this is not just a refactor and loses the logic
around $cloned_modules variable that seems to avoid doing the same
thing twice.  An explanation on the reason why it no longer is
necessary in the proposed log message would be nice.

Thanks.


>  git-submodule.sh | 31 ++++++++++++++++++++-----------
>  1 file changed, 20 insertions(+), 11 deletions(-)
>
> diff --git a/git-submodule.sh b/git-submodule.sh
> index ea3260e..7f11158 100755
> --- a/git-submodule.sh
> +++ b/git-submodule.sh
> @@ -607,6 +607,24 @@ cmd_update_recursive()
>  	fi
>  }
>  
> +cmd_update_clone()
> +{
> +	command="git checkout $subforce -q"
> +	die_msg="$(eval_gettext "Unable to checkout '\$sha1' in submodule path '\$displaypath'")"
> +	say_msg="$(eval_gettext "Submodule path '\$displaypath': checked out '\$sha1'")"
> +
> +	git submodule--helper clone ${GIT_QUIET:+--quiet} ${prefix:+--prefix "$prefix"} --path "$sm_path" --name "$name" --url "$url" "$reference" "$depth" || exit
> +
> +	if (clear_local_git_env; cd "$sm_path" && $command "$sha1")
> +	then
> +		say "$say_msg"
> +	else
> +		err="${err};$die_msg"
> +		return
> +	fi
> +	cmd_update_recursive
> +}
> +
>  #
>  # Update each submodule path to correct revision, using clone and checkout as needed
>  #
> @@ -680,7 +698,6 @@ cmd_update()
>  		cmd_init "--" "$@" || return
>  	fi
>  
> -	cloned_modules=
>  	git submodule--helper list --prefix "$wt_prefix" "$@" | {
>  	err=
>  	while read mode sha1 stage sm_path
> @@ -725,9 +742,8 @@ Maybe you want to use 'update --init'?")"
>  
>  		if ! test -d "$sm_path"/.git && ! test -f "$sm_path"/.git
>  		then
> -			git submodule--helper clone ${GIT_QUIET:+--quiet} --prefix "$prefix" --path "$sm_path" --name "$name" --url "$url" "$reference" "$depth" || exit
> -			cloned_modules="$cloned_modules;$name"
> -			subsha1=
> +			cmd_update_clone
> +			continue
>  		else
>  			subsha1=$(clear_local_git_env; cd "$sm_path" &&
>  				git rev-parse --verify HEAD) ||
> @@ -767,13 +783,6 @@ Maybe you want to use 'update --init'?")"
>  				die "$(eval_gettext "Unable to fetch in submodule path '\$displaypath'")"
>  			fi
>  
> -			# Is this something we just cloned?
> -			case ";$cloned_modules;" in
> -			*";$name;"*)
> -				# then there is no local change to integrate
> -				update_module=checkout ;;
> -			esac
> -
>  			must_die_on_failure=
>  			case "$update_module" in
>  			checkout)

^ permalink raw reply	[flat|nested] 39+ messages in thread

* [PATCHv5] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-23 17:54       ` Junio C Hamano
@ 2015-09-23 23:41         ` Stefan Beller
  2015-09-24  2:17           ` Junio C Hamano
  0 siblings, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-23 23:41 UTC (permalink / raw)
  To: git
  Cc: ramsay, jacob.keller, peff, gitster, jrnieder,
	johannes.schindelin, Jens.Lehmann, vlovich, sunshine,
	Stefan Beller

Signed-off-by: Stefan Beller <sbeller@google.com>
---

I agree the commit 7087c5f3 (SQUASH??? on top of run-command: add an
asynchronous parallel child processor) makes sense; I arrived at the same
patch after adding in the feedback.

However I have the difference below, which I think could also be squashed in.
when we operate under the assumption that the CPUs are the bottleneck in
the case of parallelism.

 run-command.c | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)

diff --git a/run-command.c b/run-command.c
index b6d8b39..e9722ff 100644
--- a/run-command.c
+++ b/run-command.c
@@ -1107,8 +1107,13 @@ static void pp_collect_finished(struct parallel_processes *pp)
 	}
 }
 
-
-#define SPAWN_CAP (pp.max_processes + 1) /* spawn as many as possible */
+/*
+ * Throttle spawning children when starting the asynchronous processing,
+ * such that the first process to produce output is not slowed down to
+ * enable the fastest early feedback to the user.
+ */
+#define SPAWN_CAP (pp.max_processes + 1 < online_cpus() - 1 ? \
+		   pp.max_processes + 1 : online_cpus() - 1)
 
 int run_processes_parallel(int n, void *data,
 			   get_next_task_fn get_next_task,
-- 
2.5.0.273.g6fa2560.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* Re: [PATCHv5] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-23 23:41         ` [PATCHv5] Another squash on " Stefan Beller
@ 2015-09-24  2:17           ` Junio C Hamano
  2015-09-24 21:13             ` [PATCH 0/2] " Stefan Beller
  0 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-24  2:17 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, sunshine

Stefan Beller <sbeller@google.com> writes:

> I agree the commit 7087c5f3 (SQUASH??? on top of run-command: add an
> asynchronous parallel child processor) makes sense; I arrived at the same
> patch after adding in the feedback.

I dunno.  As I said, while a small part of that commit is necessary
to be squashed into [6/14] (i.e. the "failed to start" bugfix and
redefinition of the return value of start_one()), the remainder of
the commit is primarily to illustrate possible future enhancements
that the basic structure of your code must support, so that we can
get the fundamentals right.  Each of the actual "possible future
enhancements" may or may not be a good idea and there is nothing
that back it up.  In such a vacuum, I'd prefer to leave them simple
and avoid starting performance tuning prematurely.

One thing that is sort-of backed up already by your "cap to 4"
experiment is that some sort of slow-start ramping up is far better
than letting thundering herd stampede, so I am OK if we kept that
SPAWN_CAP part of the commit.

But even then, we do not know if tying that cap to online_cpu() is a
good idea.  Neither of us have a good argument backed by data on it.

It is tempting to imagine, when you have N cores on an otherwise
idle box, the setting of SPAWN_CAP shouldn't make much difference to
how well the first child process makes its initial progress as long
as it does not exceed the number of idle cores N-1 you have at hand.

But that assumes that the task is CPU bound and you have infinite
memory bandwidth.  Once the task needs a lot of disk bandwidth to
make its initial progress, which certainly is the case for fetch,
the first child that is spawned together with (SPAWN_CAP-1) other
processes would be competing for the shared resource, and having
more online_cpus() would not help you.

If we are not doing analysis that takes into such factors (and it is
way too premature for us to be tuning), even "online_cpu() - 1" is
unnecessarily too complex than a hardcoded small number (say, "2",
or even "1").

The same thing can be said for the output_timeout selection.  "Do
not get stuck for too long until we have fully ramped up.  Do not
spin too frequently when there is no more room for a new child" was
something I came up out of thin air as an example of something we
might want to do, and I did write such a code in that commit, but
that was primarily done so that you can clearly see that a better
design would be to allow the caller, i.e. the scheduling loop,
specify output_timeout to buffer_stderr(), and to keep the latter a
"dumb" helper that can be controlled by a more smart caller (as
opposed to hiding such a logic in buffer_stderr() and have a "dumb"
driver call it).  The actual output_timeout computation logic is not
well thought out---it may even turn out to be that we are better off
if we lengthened the timeout before we have fully ramped up, to
encourage the first process to produce some output before we give
chance to other new processes to be spawned in the later round.

So for that change, while I think adding that parameter to
buffer_stderr() is something we would want to keep, I'd prefer to
keep the caller simpler by always passing a hardcoded 100 in the
initial version, before we start tuning.  And I do not think we want
to start tuning before building a solid foundation to tune.

In short, if I were amending that SQUASH??? commit, I'd probably be
making it do less, not more, than what it does, something along the
line of the attached.

 run-command.c | 15 ++++-----------
 1 file changed, 4 insertions(+), 11 deletions(-)

diff --git a/run-command.c b/run-command.c
index b6d8b39..829b6fe 100644
--- a/run-command.c
+++ b/run-command.c
@@ -1108,20 +1108,19 @@ static void pp_collect_finished(struct parallel_processes *pp)
 }
 
 
-#define SPAWN_CAP (pp.max_processes + 1) /* spawn as many as possible */
-
 int run_processes_parallel(int n, void *data,
 			   get_next_task_fn get_next_task,
 			   start_failure_fn start_failure,
 			   return_value_fn return_value)
 {
 	struct parallel_processes pp;
-	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
 
+	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
 	while (1) {
-		int no_more_task, cnt, output_timeout;
+		int no_more_task, cnt, output_timeout = 100;
+		int spawn_cap = 2;
 
-		for (cnt = SPAWN_CAP, no_more_task = 0;
+		for (cnt = spawn_cap, no_more_task = 0;
 		     cnt && pp.nr_processes < pp.max_processes;
 		     cnt--) {
 			if (!pp_start_one(&pp)) {
@@ -1132,12 +1131,6 @@ int run_processes_parallel(int n, void *data,
 
 		if (no_more_task && !pp.nr_processes)
 			break;
-		if (!cnt)
-			output_timeout = 50;
-		else if (pp.nr_processes < pp.max_processes)
-			output_timeout = 100;
-		else
-			output_timeout = 1000;
 		pp_buffer_stderr(&pp, output_timeout);
 
 		pp_output(&pp);

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-24  2:17           ` Junio C Hamano
@ 2015-09-24 21:13             ` Stefan Beller
  2015-09-24 21:13               ` [PATCH 2/2] SQUASH for "fetch_populated_submodules: use new parallel job processing" Stefan Beller
                                 ` (2 more replies)
  0 siblings, 3 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-24 21:13 UTC (permalink / raw)
  To: git, gitster
  Cc: ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine, Stefan Beller

Here is another proposal for squashing.
It's basically both squash proposals by Junio, plus another issues I think
should be done right from the beginning.

 [added by sb:]
 * If you do not die() in start_failure_fn or return_value_fn, you
   don't want to write to stderr directly as you would destroy the fine
   ordering of the processes output. So make the err strbuf available in
   both these functions, and make sure the strbuf is appended to the
   buffered output in both cases.

Stefan Beller (2):
  SQUASH???
  SQUASH for "fetch_populated_submodules: use new parallel job
    processing"

-- 
2.5.0.273.g6fa2560.dirty

^ permalink raw reply	[flat|nested] 39+ messages in thread

* [PATCH 2/2] SQUASH for "fetch_populated_submodules: use new parallel job processing"
  2015-09-24 21:13             ` [PATCH 0/2] " Stefan Beller
@ 2015-09-24 21:13               ` Stefan Beller
  2015-09-24 21:13               ` [PATCH 1/2] SQUASH??? Stefan Beller
  2015-09-25  1:08               ` [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor Junio C Hamano
  2 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-24 21:13 UTC (permalink / raw)
  To: git, gitster
  Cc: ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine, Stefan Beller

This fixes the function signature in the first user of the async run processor.

Signed-off-by: Stefan Beller <sbeller@google.com>
---
 submodule.c | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)

diff --git a/submodule.c b/submodule.c
index d7c7a6e..2c4396b 100644
--- a/submodule.c
+++ b/submodule.c
@@ -630,13 +630,15 @@ struct submodule_parallel_fetch {
 int get_next_submodule(void *data, struct child_process *cp,
 		       struct strbuf *err);
 
-void handle_submodule_fetch_start_err(void *data, struct child_process *cp, struct strbuf *err)
+void handle_submodule_fetch_start_err(void *data, struct child_process *cp,
+				      struct strbuf *err)
 {
 	struct submodule_parallel_fetch *spf = data;
 	spf->result = 1;
 }
 
-void handle_submodule_fetch_finish( void *data, struct child_process *cp, int retvalue)
+void handle_submodule_fetch_finish(void *data, struct child_process *cp,
+				   struct strbuf *err, int retvalue)
 {
 	struct submodule_parallel_fetch *spf = data;
 
-- 
2.5.0.273.g6fa2560.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* [PATCH 1/2] SQUASH???
  2015-09-24 21:13             ` [PATCH 0/2] " Stefan Beller
  2015-09-24 21:13               ` [PATCH 2/2] SQUASH for "fetch_populated_submodules: use new parallel job processing" Stefan Beller
@ 2015-09-24 21:13               ` Stefan Beller
  2015-09-25  0:49                 ` Junio C Hamano
  2015-09-25  1:08               ` [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor Junio C Hamano
  2 siblings, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-24 21:13 UTC (permalink / raw)
  To: git, gitster
  Cc: ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine, Stefan Beller

This is a mixed bag of squashes.

 * pp_start_one() stuffed unborn child to the list of children when
   start_command() failed and start_failure() did not die(); return
   to the caller without corrupting children[] list in this case.

 * make poll(2) timeout used in pp_buffer_stderr() configurable by
   the scheduling loop.

 * allow slow-start of the whole process, so that we do not spawn
   tons of processes before starting to read from any of them, to
   give a better first-byte latency.

 * fix the semantics of the value returned from pp_start_one() and
   adjust the scheduling loop for it.

[added by sb:]
 * If you do not die() in start_failure_fn or return_value_fn, you
   don't want to write to stderr directly as you would destroy the fine
   ordering of the processes output. So make the err strbuf available in
   both these functions, and make sure the strbuf is appended to the
   buffered output in both cases

Signed-off-by: Junio C Hamano <gitster@pobox.com>
Signed-off-by: Stefan Beller <sbeller@google.com>
---
 run-command.c | 43 ++++++++++++++++++++++++++++++-------------
 run-command.h |  1 +
 2 files changed, 31 insertions(+), 13 deletions(-)

diff --git a/run-command.c b/run-command.c
index 494e1f8..0d22291 100644
--- a/run-command.c
+++ b/run-command.c
@@ -907,6 +907,7 @@ void default_start_failure(void *data,
 
 void default_return_value(void *data,
 			  struct child_process *cp,
+			  struct strbuf *err,
 			  int result)
 {
 	int i;
@@ -977,7 +978,7 @@ static void set_nonblocking(int fd)
 			"output will be degraded");
 }
 
-/* returns 1 if a process was started, 0 otherwise */
+/* return 0 if get_next_task() ran out of things to do, non-zero otherwise */
 static int pp_start_one(struct parallel_processes *pp)
 {
 	int i;
@@ -991,26 +992,30 @@ static int pp_start_one(struct parallel_processes *pp)
 	if (!pp->get_next_task(pp->data,
 			       &pp->children[i].process,
 			       &pp->children[i].err))
-		return 1;
+		return 0;
 
-	if (start_command(&pp->children[i].process))
+	if (start_command(&pp->children[i].process)) {
 		pp->start_failure(pp->data,
 				  &pp->children[i].process,
 				  &pp->children[i].err);
+		strbuf_addbuf(&pp->buffered_output, &pp->children[i].err);
+		strbuf_reset(&pp->children[i].err);
+		return -1;
+	}
 
 	set_nonblocking(pp->children[i].process.err);
 
 	pp->nr_processes++;
 	pp->children[i].in_use = 1;
 	pp->pfd[i].fd = pp->children[i].process.err;
-	return 0;
+	return 1;
 }
 
-static void pp_buffer_stderr(struct parallel_processes *pp)
+static void pp_buffer_stderr(struct parallel_processes *pp, int output_timeout)
 {
 	int i;
 
-	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
+	while ((i = poll(pp->pfd, pp->max_processes, output_timeout)) < 0) {
 		if (errno == EINTR)
 			continue;
 		pp_cleanup(pp);
@@ -1069,7 +1074,8 @@ static void pp_collect_finished(struct parallel_processes *pp)
 			error("waitpid is confused (%s)",
 			      pp->children[i].process.argv[0]);
 
-		pp->return_value(pp->data, &pp->children[i].process, code);
+		pp->return_value(pp->data, &pp->children[i].process,
+				 &pp->children[i].err, code);
 
 		argv_array_clear(&pp->children[i].process.args);
 		argv_array_clear(&pp->children[i].process.env_array);
@@ -1111,15 +1117,26 @@ int run_processes_parallel(int n, void *data,
 			   return_value_fn return_value)
 {
 	struct parallel_processes pp;
-	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
 
+	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
 	while (1) {
-		while (pp.nr_processes < pp.max_processes &&
-		       !pp_start_one(&pp))
-			; /* nothing */
-		if (!pp.nr_processes)
+		int no_more_task, cnt;
+		int output_timeout = 100;
+		int spawn_cap = 4;
+
+		for (cnt = spawn_cap, no_more_task = 0;
+		     cnt && pp.nr_processes < pp.max_processes;
+		     cnt--) {
+			if (!pp_start_one(&pp)) {
+				no_more_task = 1;
+				break;
+			}
+		}
+
+		if (no_more_task && !pp.nr_processes)
 			break;
-		pp_buffer_stderr(&pp);
+		pp_buffer_stderr(&pp, output_timeout);
+
 		pp_output(&pp);
 		pp_collect_finished(&pp);
 	}
diff --git a/run-command.h b/run-command.h
index 3807fd1..f7035cb 100644
--- a/run-command.h
+++ b/run-command.h
@@ -138,6 +138,7 @@ typedef void (*start_failure_fn)(void *data,
 
 typedef void (*return_value_fn)(void *data,
 				struct child_process *cp,
+				struct strbuf *err,
 				int result);
 
 /**
-- 
2.5.0.273.g6fa2560.dirty

^ permalink raw reply related	[flat|nested] 39+ messages in thread

* Re: [PATCH 1/2] SQUASH???
  2015-09-24 21:13               ` [PATCH 1/2] SQUASH??? Stefan Beller
@ 2015-09-25  0:49                 ` Junio C Hamano
  2015-09-25  1:09                   ` Junio C Hamano
  2015-09-25 17:52                   ` Stefan Beller
  0 siblings, 2 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25  0:49 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine

Stefan Beller <sbeller@google.com> writes:

>  * If you do not die() in start_failure_fn or return_value_fn, you
>    don't want to write to stderr directly as you would destroy the fine
>    ordering of the processes output. So make the err strbuf available in
>    both these functions, and make sure the strbuf is appended to the
>    buffered output in both cases

I think that is a sensible change.  Don't we want the same for the
other failure handler, though.  Capture any message from it and
append it to the output of the process that just finished, or
something?

By the way, I understand that these two are solely for early review
and I'll be getting them as either new patches or part of updated
patches in the next reroll (i.e. you are not expecting me to split
these apart and do "rebase -i" for you to the last-posted version).
Asking only to make sure we are on the same wavelength.

Thanks.

>
> Signed-off-by: Junio C Hamano <gitster@pobox.com>
> Signed-off-by: Stefan Beller <sbeller@google.com>
> ---
>  run-command.c | 43 ++++++++++++++++++++++++++++++-------------
>  run-command.h |  1 +
>  2 files changed, 31 insertions(+), 13 deletions(-)
>
> diff --git a/run-command.c b/run-command.c
> index 494e1f8..0d22291 100644
> --- a/run-command.c
> +++ b/run-command.c
> @@ -907,6 +907,7 @@ void default_start_failure(void *data,
>  
>  void default_return_value(void *data,
>  			  struct child_process *cp,
> +			  struct strbuf *err,
>  			  int result)
>  {
>  	int i;
> @@ -977,7 +978,7 @@ static void set_nonblocking(int fd)
>  			"output will be degraded");
>  }
>  
> -/* returns 1 if a process was started, 0 otherwise */
> +/* return 0 if get_next_task() ran out of things to do, non-zero otherwise */
>  static int pp_start_one(struct parallel_processes *pp)
>  {
>  	int i;
> @@ -991,26 +992,30 @@ static int pp_start_one(struct parallel_processes *pp)
>  	if (!pp->get_next_task(pp->data,
>  			       &pp->children[i].process,
>  			       &pp->children[i].err))
> -		return 1;
> +		return 0;
>  
> -	if (start_command(&pp->children[i].process))
> +	if (start_command(&pp->children[i].process)) {
>  		pp->start_failure(pp->data,
>  				  &pp->children[i].process,
>  				  &pp->children[i].err);
> +		strbuf_addbuf(&pp->buffered_output, &pp->children[i].err);
> +		strbuf_reset(&pp->children[i].err);
> +		return -1;
> +	}
>  
>  	set_nonblocking(pp->children[i].process.err);
>  
>  	pp->nr_processes++;
>  	pp->children[i].in_use = 1;
>  	pp->pfd[i].fd = pp->children[i].process.err;
> -	return 0;
> +	return 1;
>  }
>  
> -static void pp_buffer_stderr(struct parallel_processes *pp)
> +static void pp_buffer_stderr(struct parallel_processes *pp, int output_timeout)
>  {
>  	int i;
>  
> -	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
> +	while ((i = poll(pp->pfd, pp->max_processes, output_timeout)) < 0) {
>  		if (errno == EINTR)
>  			continue;
>  		pp_cleanup(pp);
> @@ -1069,7 +1074,8 @@ static void pp_collect_finished(struct parallel_processes *pp)
>  			error("waitpid is confused (%s)",
>  			      pp->children[i].process.argv[0]);
>  
> -		pp->return_value(pp->data, &pp->children[i].process, code);
> +		pp->return_value(pp->data, &pp->children[i].process,
> +				 &pp->children[i].err, code);
>  
>  		argv_array_clear(&pp->children[i].process.args);
>  		argv_array_clear(&pp->children[i].process.env_array);
> @@ -1111,15 +1117,26 @@ int run_processes_parallel(int n, void *data,
>  			   return_value_fn return_value)
>  {
>  	struct parallel_processes pp;
> -	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>  
> +	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>  	while (1) {
> -		while (pp.nr_processes < pp.max_processes &&
> -		       !pp_start_one(&pp))
> -			; /* nothing */
> -		if (!pp.nr_processes)
> +		int no_more_task, cnt;
> +		int output_timeout = 100;
> +		int spawn_cap = 4;
> +
> +		for (cnt = spawn_cap, no_more_task = 0;
> +		     cnt && pp.nr_processes < pp.max_processes;
> +		     cnt--) {
> +			if (!pp_start_one(&pp)) {
> +				no_more_task = 1;
> +				break;
> +			}
> +		}
> +
> +		if (no_more_task && !pp.nr_processes)
>  			break;
> -		pp_buffer_stderr(&pp);
> +		pp_buffer_stderr(&pp, output_timeout);
> +
>  		pp_output(&pp);
>  		pp_collect_finished(&pp);
>  	}
> diff --git a/run-command.h b/run-command.h
> index 3807fd1..f7035cb 100644
> --- a/run-command.h
> +++ b/run-command.h
> @@ -138,6 +138,7 @@ typedef void (*start_failure_fn)(void *data,
>  
>  typedef void (*return_value_fn)(void *data,
>  				struct child_process *cp,
> +				struct strbuf *err,
>  				int result);
>  
>  /**

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-24 21:13             ` [PATCH 0/2] " Stefan Beller
  2015-09-24 21:13               ` [PATCH 2/2] SQUASH for "fetch_populated_submodules: use new parallel job processing" Stefan Beller
  2015-09-24 21:13               ` [PATCH 1/2] SQUASH??? Stefan Beller
@ 2015-09-25  1:08               ` Junio C Hamano
  2015-09-25 18:56                 ` Stefan Beller
  2 siblings, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25  1:08 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine

Stefan Beller <sbeller@google.com> writes:

>  * If you do not die() in start_failure_fn or return_value_fn, you
>    don't want to write to stderr directly as you would destroy the fine
>    ordering of the processes output. So make the err strbuf available in
>    both these functions, and make sure the strbuf is appended to the
>    buffered output in both cases.

Another thing I noticed after re-reading the above is that we shared
the thinking that dying in these is _the_ normal thing to do and
continuing is an advanced and/or wierd setting.

And I think it is wrong.  Suppose after spawning 15 tasks and while
they are still running, you start the 16th one and it fails to stop.
If your start-failure called die() to kill the controller, what
happens to the 15 tasks that are already running?

I think two sensible choices that start-failure and return-value can
make are

 (1) This one task failed, but that is OK.  Please let the other
     tasks run [*1*].

 (2) There is something seriously wrong with the whole world and I
     declare an emergency.  Please kill the other ones and exit.

Dying in these callbacks do not achieve neither.  Perhaps make these
two functions return bool (or enum if you already know a third
sensible option, but otherwise bool is fine and the person who
discovers the need for the third will turn it into enum) to signal
which one of these two behaviours it wants?

And the default handlers should stop dying, of course.


[Footnote]

*1* Because start-failure gets pp, it can even leave a note in it to
    ask the next invocation of get-next to retry it if it chooses
    to.  At this point in the design cycle, all we need to do is to
    make sure that kind of advanced usage is possible with this
    parallel-run-command API.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 1/2] SQUASH???
  2015-09-25  0:49                 ` Junio C Hamano
@ 2015-09-25  1:09                   ` Junio C Hamano
  2015-09-25 17:52                   ` Stefan Beller
  1 sibling, 0 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25  1:09 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, ramsay, jacob.keller, peff, jrnieder, johannes.schindelin,
	Jens.Lehmann, vlovich, ericsunshine

Junio C Hamano <gitster@pobox.com> writes:

> I think that is a sensible change.  Don't we want the same for the
> other failure handler, though.  Capture any message from it and
> append it to the output of the process that just finished, or
> something?

Ah, that is already done.  Scratch that "don't we want" part.

>
> By the way, I understand that these two are solely for early review
> and I'll be getting them as either new patches or part of updated
> patches in the next reroll (i.e. you are not expecting me to split
> these apart and do "rebase -i" for you to the last-posted version).
> Asking only to make sure we are on the same wavelength.
>
> Thanks.
>
>>
>> Signed-off-by: Junio C Hamano <gitster@pobox.com>
>> Signed-off-by: Stefan Beller <sbeller@google.com>
>> ---
>>  run-command.c | 43 ++++++++++++++++++++++++++++++-------------
>>  run-command.h |  1 +
>>  2 files changed, 31 insertions(+), 13 deletions(-)
>>
>> diff --git a/run-command.c b/run-command.c
>> index 494e1f8..0d22291 100644
>> --- a/run-command.c
>> +++ b/run-command.c
>> @@ -907,6 +907,7 @@ void default_start_failure(void *data,
>>  
>>  void default_return_value(void *data,
>>  			  struct child_process *cp,
>> +			  struct strbuf *err,
>>  			  int result)
>>  {
>>  	int i;
>> @@ -977,7 +978,7 @@ static void set_nonblocking(int fd)
>>  			"output will be degraded");
>>  }
>>  
>> -/* returns 1 if a process was started, 0 otherwise */
>> +/* return 0 if get_next_task() ran out of things to do, non-zero otherwise */
>>  static int pp_start_one(struct parallel_processes *pp)
>>  {
>>  	int i;
>> @@ -991,26 +992,30 @@ static int pp_start_one(struct parallel_processes *pp)
>>  	if (!pp->get_next_task(pp->data,
>>  			       &pp->children[i].process,
>>  			       &pp->children[i].err))
>> -		return 1;
>> +		return 0;
>>  
>> -	if (start_command(&pp->children[i].process))
>> +	if (start_command(&pp->children[i].process)) {
>>  		pp->start_failure(pp->data,
>>  				  &pp->children[i].process,
>>  				  &pp->children[i].err);
>> +		strbuf_addbuf(&pp->buffered_output, &pp->children[i].err);
>> +		strbuf_reset(&pp->children[i].err);
>> +		return -1;
>> +	}
>>  
>>  	set_nonblocking(pp->children[i].process.err);
>>  
>>  	pp->nr_processes++;
>>  	pp->children[i].in_use = 1;
>>  	pp->pfd[i].fd = pp->children[i].process.err;
>> -	return 0;
>> +	return 1;
>>  }
>>  
>> -static void pp_buffer_stderr(struct parallel_processes *pp)
>> +static void pp_buffer_stderr(struct parallel_processes *pp, int output_timeout)
>>  {
>>  	int i;
>>  
>> -	while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
>> +	while ((i = poll(pp->pfd, pp->max_processes, output_timeout)) < 0) {
>>  		if (errno == EINTR)
>>  			continue;
>>  		pp_cleanup(pp);
>> @@ -1069,7 +1074,8 @@ static void pp_collect_finished(struct parallel_processes *pp)
>>  			error("waitpid is confused (%s)",
>>  			      pp->children[i].process.argv[0]);
>>  
>> -		pp->return_value(pp->data, &pp->children[i].process, code);
>> +		pp->return_value(pp->data, &pp->children[i].process,
>> +				 &pp->children[i].err, code);
>>  
>>  		argv_array_clear(&pp->children[i].process.args);
>>  		argv_array_clear(&pp->children[i].process.env_array);
>> @@ -1111,15 +1117,26 @@ int run_processes_parallel(int n, void *data,
>>  			   return_value_fn return_value)
>>  {
>>  	struct parallel_processes pp;
>> -	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>>  
>> +	pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>>  	while (1) {
>> -		while (pp.nr_processes < pp.max_processes &&
>> -		       !pp_start_one(&pp))
>> -			; /* nothing */
>> -		if (!pp.nr_processes)
>> +		int no_more_task, cnt;
>> +		int output_timeout = 100;
>> +		int spawn_cap = 4;
>> +
>> +		for (cnt = spawn_cap, no_more_task = 0;
>> +		     cnt && pp.nr_processes < pp.max_processes;
>> +		     cnt--) {
>> +			if (!pp_start_one(&pp)) {
>> +				no_more_task = 1;
>> +				break;
>> +			}
>> +		}
>> +
>> +		if (no_more_task && !pp.nr_processes)
>>  			break;
>> -		pp_buffer_stderr(&pp);
>> +		pp_buffer_stderr(&pp, output_timeout);
>> +
>>  		pp_output(&pp);
>>  		pp_collect_finished(&pp);
>>  	}
>> diff --git a/run-command.h b/run-command.h
>> index 3807fd1..f7035cb 100644
>> --- a/run-command.h
>> +++ b/run-command.h
>> @@ -138,6 +138,7 @@ typedef void (*start_failure_fn)(void *data,
>>  
>>  typedef void (*return_value_fn)(void *data,
>>  				struct child_process *cp,
>> +				struct strbuf *err,
>>  				int result);
>>  
>>  /**

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 1/2] SQUASH???
  2015-09-25  0:49                 ` Junio C Hamano
  2015-09-25  1:09                   ` Junio C Hamano
@ 2015-09-25 17:52                   ` Stefan Beller
  2015-09-25 17:56                     ` Junio C Hamano
  1 sibling, 1 reply; 39+ messages in thread
From: Stefan Beller @ 2015-09-25 17:52 UTC (permalink / raw)
  To: Junio C Hamano
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

On Thu, Sep 24, 2015 at 5:49 PM, Junio C Hamano <gitster@pobox.com> wrote:
> Stefan Beller <sbeller@google.com> writes:
>
>>  * If you do not die() in start_failure_fn or return_value_fn, you
>>    don't want to write to stderr directly as you would destroy the fine
>>    ordering of the processes output. So make the err strbuf available in
>>    both these functions, and make sure the strbuf is appended to the
>>    buffered output in both cases
>
> I think that is a sensible change.  Don't we want the same for the
> other failure handler, though.  Capture any message from it and
> append it to the output of the process that just finished, or
> something?
>
> By the way, I understand that these two are solely for early review
> and I'll be getting them as either new patches or part of updated
> patches in the next reroll (i.e. you are not expecting me to split
> these apart and do "rebase -i" for you to the last-posted version).
> Asking only to make sure we are on the same wavelength.

Sure. I just wanted to point out the details instead of resending the series.
I'll do a resend later today, hoping to get all issues addressed.

Thanks,
Stefan


>
> Thanks.
>
>>
>> Signed-off-by: Junio C Hamano <gitster@pobox.com>
>> Signed-off-by: Stefan Beller <sbeller@google.com>
>> ---
>>  run-command.c | 43 ++++++++++++++++++++++++++++++-------------
>>  run-command.h |  1 +
>>  2 files changed, 31 insertions(+), 13 deletions(-)
>>
>> diff --git a/run-command.c b/run-command.c
>> index 494e1f8..0d22291 100644
>> --- a/run-command.c
>> +++ b/run-command.c
>> @@ -907,6 +907,7 @@ void default_start_failure(void *data,
>>
>>  void default_return_value(void *data,
>>                         struct child_process *cp,
>> +                       struct strbuf *err,
>>                         int result)
>>  {
>>       int i;
>> @@ -977,7 +978,7 @@ static void set_nonblocking(int fd)
>>                       "output will be degraded");
>>  }
>>
>> -/* returns 1 if a process was started, 0 otherwise */
>> +/* return 0 if get_next_task() ran out of things to do, non-zero otherwise */
>>  static int pp_start_one(struct parallel_processes *pp)
>>  {
>>       int i;
>> @@ -991,26 +992,30 @@ static int pp_start_one(struct parallel_processes *pp)
>>       if (!pp->get_next_task(pp->data,
>>                              &pp->children[i].process,
>>                              &pp->children[i].err))
>> -             return 1;
>> +             return 0;
>>
>> -     if (start_command(&pp->children[i].process))
>> +     if (start_command(&pp->children[i].process)) {
>>               pp->start_failure(pp->data,
>>                                 &pp->children[i].process,
>>                                 &pp->children[i].err);
>> +             strbuf_addbuf(&pp->buffered_output, &pp->children[i].err);
>> +             strbuf_reset(&pp->children[i].err);
>> +             return -1;
>> +     }
>>
>>       set_nonblocking(pp->children[i].process.err);
>>
>>       pp->nr_processes++;
>>       pp->children[i].in_use = 1;
>>       pp->pfd[i].fd = pp->children[i].process.err;
>> -     return 0;
>> +     return 1;
>>  }
>>
>> -static void pp_buffer_stderr(struct parallel_processes *pp)
>> +static void pp_buffer_stderr(struct parallel_processes *pp, int output_timeout)
>>  {
>>       int i;
>>
>> -     while ((i = poll(pp->pfd, pp->max_processes, 100)) < 0) {
>> +     while ((i = poll(pp->pfd, pp->max_processes, output_timeout)) < 0) {
>>               if (errno == EINTR)
>>                       continue;
>>               pp_cleanup(pp);
>> @@ -1069,7 +1074,8 @@ static void pp_collect_finished(struct parallel_processes *pp)
>>                       error("waitpid is confused (%s)",
>>                             pp->children[i].process.argv[0]);
>>
>> -             pp->return_value(pp->data, &pp->children[i].process, code);
>> +             pp->return_value(pp->data, &pp->children[i].process,
>> +                              &pp->children[i].err, code);
>>
>>               argv_array_clear(&pp->children[i].process.args);
>>               argv_array_clear(&pp->children[i].process.env_array);
>> @@ -1111,15 +1117,26 @@ int run_processes_parallel(int n, void *data,
>>                          return_value_fn return_value)
>>  {
>>       struct parallel_processes pp;
>> -     pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>>
>> +     pp_init(&pp, n, data, get_next_task, start_failure, return_value);
>>       while (1) {
>> -             while (pp.nr_processes < pp.max_processes &&
>> -                    !pp_start_one(&pp))
>> -                     ; /* nothing */
>> -             if (!pp.nr_processes)
>> +             int no_more_task, cnt;
>> +             int output_timeout = 100;
>> +             int spawn_cap = 4;
>> +
>> +             for (cnt = spawn_cap, no_more_task = 0;
>> +                  cnt && pp.nr_processes < pp.max_processes;
>> +                  cnt--) {
>> +                     if (!pp_start_one(&pp)) {
>> +                             no_more_task = 1;
>> +                             break;
>> +                     }
>> +             }
>> +
>> +             if (no_more_task && !pp.nr_processes)
>>                       break;
>> -             pp_buffer_stderr(&pp);
>> +             pp_buffer_stderr(&pp, output_timeout);
>> +
>>               pp_output(&pp);
>>               pp_collect_finished(&pp);
>>       }
>> diff --git a/run-command.h b/run-command.h
>> index 3807fd1..f7035cb 100644
>> --- a/run-command.h
>> +++ b/run-command.h
>> @@ -138,6 +138,7 @@ typedef void (*start_failure_fn)(void *data,
>>
>>  typedef void (*return_value_fn)(void *data,
>>                               struct child_process *cp,
>> +                             struct strbuf *err,
>>                               int result);
>>
>>  /**

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 1/2] SQUASH???
  2015-09-25 17:52                   ` Stefan Beller
@ 2015-09-25 17:56                     ` Junio C Hamano
  0 siblings, 0 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25 17:56 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Stefan Beller <sbeller@google.com> writes:

> Sure. I just wanted to point out the details instead of resending the series.
> I'll do a resend later today, hoping to get all issues addressed.

Thanks.  For something this small, unless there are many small
pieces that need to be picked apart, I do not terribly mind to do
the squashing myself (after all that is what I do every day, prepare
a squash while queuing, do the integration testing and send the
squash out to make sure the original author agrees with it).

I just wanted to make sure that we won't be in a situation where
I am waiting forever expecting you to reroll while you are expecting
me to do the squashing, resulting in no progress getting made until
either one of us starts wondering what is going on.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-25  1:08               ` [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor Junio C Hamano
@ 2015-09-25 18:56                 ` Stefan Beller
  2015-09-25 19:04                   ` Junio C Hamano
  2015-09-25 19:32                   ` Junio C Hamano
  0 siblings, 2 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-25 18:56 UTC (permalink / raw)
  To: Junio C Hamano
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

On Thu, Sep 24, 2015 at 6:08 PM, Junio C Hamano <gitster@pobox.com> wrote:
> Stefan Beller <sbeller@google.com> writes:
>
>>  * If you do not die() in start_failure_fn or return_value_fn, you
>>    don't want to write to stderr directly as you would destroy the fine
>>    ordering of the processes output. So make the err strbuf available in
>>    both these functions, and make sure the strbuf is appended to the
>>    buffered output in both cases.
>
> Another thing I noticed after re-reading the above is that we shared
> the thinking that dying in these is _the_ normal thing to do and
> continuing is an advanced and/or wierd setting.
>
> And I think it is wrong.  Suppose after spawning 15 tasks and while
> they are still running, you start the 16th one and it fails to stop.
> If your start-failure called die() to kill the controller, what
> happens to the 15 tasks that are already running?
>
> I think two sensible choices that start-failure and return-value can
> make are
>
>  (1) This one task failed, but that is OK.  Please let the other
>      tasks run [*1*].
>
>  (2) There is something seriously wrong with the whole world and I
>      declare an emergency.  Please kill the other ones and exit.

  (3) There is something wrong, such that I cannot finish my
      job, but I know the other 15 processes help towards the goal,
      so I want to let them live on until they are done. E.g: fetch submodules
      may want to take this strategy if it fails to start another sub
process fetching.

By having a return value indicating which strategy you want to pursue here,
we're making the design choice to have everything done monolithically
inside the pp machinery.

We could also offer more access to the pp machinery and an implementation for
(2) might look like this:

static void fictious_start_failure(void *data,
                                void *pp,
                                struct child_process *cp,
                                struct strbuf *err)
{
        struct mydata *m = data;

        if (m->failstrategy == 1)
                ; /* nothing here */
        else if (m->failstrategy == 2)
                killall_children(pp);
        else if (m->failstrategy == 3) {
                m->stop_scheduling_new_tasks = 1;
                redirect_children_to_dev_null(pp);
        else
                ...
}

By having the pointer to the pp struct passed around, we allow
for adding new callback functions to be added later to the
pp machinery, which may not be expressed via a return code.

>
> Dying in these callbacks do not achieve neither.  Perhaps make these
> two functions return bool (or enum if you already know a third
> sensible option, but otherwise bool is fine and the person who
> discovers the need for the third will turn it into enum) to signal
> which one of these two behaviours it wants?
>
> And the default handlers should stop dying, of course.
>
>
> [Footnote]
>
> *1* Because start-failure gets pp, it can even leave a note in it to
>     ask the next invocation of get-next to retry it if it chooses
>     to.  At this point in the design cycle, all we need to do is to
>     make sure that kind of advanced usage is possible with this
>     parallel-run-command API.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-25 18:56                 ` Stefan Beller
@ 2015-09-25 19:04                   ` Junio C Hamano
  2015-09-25 19:19                     ` Stefan Beller
  2015-09-25 19:32                   ` Junio C Hamano
  1 sibling, 1 reply; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25 19:04 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Stefan Beller <sbeller@google.com> writes:

>> I think two sensible choices that start-failure and return-value can
>> make are
>>
>>  (1) This one task failed, but that is OK.  Please let the other
>>      tasks run [*1*].
>>
>>  (2) There is something seriously wrong with the whole world and I
>>      declare an emergency.  Please kill the other ones and exit.
>
>   (3) There is something wrong, such that I cannot finish my
>       job, but I know the other 15 processes help towards the goal,
>       so I want to let them live on until they are done. E.g: fetch submodules
>       may want to take this strategy if it fails to start another sub
> process fetching.

How is that different from (1)?  Do you meann "let other ones that
are already running continue, but do not spawn any new ones?"

> We could also offer more access to the pp machinery and an implementation for
> (2) might look like this:
> ...
> By having the pointer to the pp struct passed around, we allow
> for adding new callback functions to be added later to the
> pp machinery, which may not be expressed via a return code.

What you are suggesting would lead to the same "different smart
participants making decisions locally, so you need to run around and
follow all the detailed codepaths to understand what is going on"
design.

I was hoping that we have already passed discussing that stage.

The whole point of that "SQUASH???" commit was to correct the design
of the overall structure so that we make the central dispatcher that
uses bunch of "dumb" helpers (that do not make policy decisions
locally on their own) as the single place you need to read in order
to understand the logic.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-25 19:04                   ` Junio C Hamano
@ 2015-09-25 19:19                     ` Stefan Beller
  0 siblings, 0 replies; 39+ messages in thread
From: Stefan Beller @ 2015-09-25 19:19 UTC (permalink / raw)
  To: Junio C Hamano
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

On Fri, Sep 25, 2015 at 12:04 PM, Junio C Hamano <gitster@pobox.com> wrote:
> Stefan Beller <sbeller@google.com> writes:
>
>>> I think two sensible choices that start-failure and return-value can
>>> make are
>>>
>>>  (1) This one task failed, but that is OK.  Please let the other
>>>      tasks run [*1*].
>>>
>>>  (2) There is something seriously wrong with the whole world and I
>>>      declare an emergency.  Please kill the other ones and exit.
>>
>>   (3) There is something wrong, such that I cannot finish my
>>       job, but I know the other 15 processes help towards the goal,
>>       so I want to let them live on until they are done. E.g: fetch submodules
>>       may want to take this strategy if it fails to start another sub
>> process fetching.
>
> How is that different from (1)?  Do you meann "let other ones that
> are already running continue, but do not spawn any new ones?"

Right. "Phasing out" before gracefully dying. Well to achieve this,
we don't need more options here. (Just set a flag to not return more
work in get_next_task).

>
>> We could also offer more access to the pp machinery and an implementation for
>> (2) might look like this:
>> ...
>> By having the pointer to the pp struct passed around, we allow
>> for adding new callback functions to be added later to the
>> pp machinery, which may not be expressed via a return code.
>
> What you are suggesting would lead to the same "different smart
> participants making decisions locally, so you need to run around and
> follow all the detailed codepaths to understand what is going on"
> design.
>
> I was hoping that we have already passed discussing that stage.
>
> The whole point of that "SQUASH???" commit was to correct the design
> of the overall structure so that we make the central dispatcher that
> uses bunch of "dumb" helpers (that do not make policy decisions
> locally on their own) as the single place you need to read in order
> to understand the logic.

And I thought the discussion there was focused on the internals of the
dispatcher. (You need to only read one place inside _that_ file now)

This discussion would have focused on the public API side (What do we
want to hide, how do we want to hide it?), which would be different enough
for me to warrant a new discussion.

But I guess I'll just go with your suggestion for now to have the return value
indicate a full stop or keep going.

^ permalink raw reply	[flat|nested] 39+ messages in thread

* Re: [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor
  2015-09-25 18:56                 ` Stefan Beller
  2015-09-25 19:04                   ` Junio C Hamano
@ 2015-09-25 19:32                   ` Junio C Hamano
  1 sibling, 0 replies; 39+ messages in thread
From: Junio C Hamano @ 2015-09-25 19:32 UTC (permalink / raw)
  To: Stefan Beller
  Cc: git, Ramsay Jones, Jacob Keller, Jeff King, Jonathan Nieder,
	Johannes Schindelin, Jens Lehmann, Vitali Lovich, Eric Sunshine

Stefan Beller <sbeller@google.com> writes:

> We could also offer more access to the pp machinery and an implementation for
> (2) might look like this:
>
> static void fictious_start_failure(void *data,
>                                 void *pp,
>                                 struct child_process *cp,
>                                 struct strbuf *err)
> {
>         struct mydata *m = data;
>
>         if (m->failstrategy == 1)
>                 ; /* nothing here */
>         else if (m->failstrategy == 2)
>                 killall_children(pp);

That looks nice and clean in theory, but I highly doubt that it is a
good idea to have killall_children(pp) in a client code of the API
like this, especially if you are envisioning that that function
merely goes through the list of children and does kill on the
processes.  If killall_children(pp) is supplied by the API, it makes
things less insane (now it can know and keep up with the evolution
of the API implementation detail, such as how buffered output are
kept and such), but still such an API constrains the structure of
the overall scheduling loop and the helper functions that live on
the API side.  You need to make absolutely sure that calling
killall_children() is something that can be sanely done from inside
start_failure() callback, for example.

If you signal the "emergency" with a return value, the callchain on
the API side can choose to do the killing at a place it knows is
safe to do so in a more controlled way.  For example, the main loop
of the API side IIRC (I do not bother checking out 'pu' to read it
as my working tree is busy on another topic right now) is

    while (1) {
    	for (cnt = 0; cnt < 4 && we still have slots; cnt++)
		start_one();

	collect_output();
	drain_output_for_foreground();
	wait_and_flush();
    }

Imagine that you have 0 or more processes running and start another
iteration of this loop.  The first task started by start_one() fails
and the above fictitious_start_failure() calls killall_children()
itself.  What happens to the other three iteration of the inner
loop?  After existing ones are killed, it adds three more?

And to prevent that from happening, you also need to tell your
fictitious_next_task() that no more processes are desired.  The
client of the API is forced to coordinate across its multiple
callback functions.

And you did that to gain what?  One major thing I can think of is
that that way, the main scheduling loop does not have to know why
after attempting to spawn but failing, fictitious_next_task()
started saying "no more tasks".  For somebody who is coming from
"The main loop is a dumb bullet-list of things to do.  All smarts
are inside the callee", that might appear to be a good thing.

But I do not think it is a good thing at all.  That forces the
reader of the code to not just follow the API layer but even down to
its individual clients of the API to understand what is going on.

If you propagate the return from start_failure() callback upwards,
then the main scheduler would *know* that the client application
does not want any more new tasks, and it does want to abort even the
running tasks, so it will not call fictitious_next_task() in the
first place after the client application declares an "emergency
exit".

And that would make the overall logic a lot easier to follow.








	

^ permalink raw reply	[flat|nested] 39+ messages in thread

end of thread, other threads:[~2015-09-25 19:32 UTC | newest]

Thread overview: 39+ messages (download: mbox.gz / follow: Atom feed)
-- links below jump to the message on this page --
2015-09-23  1:45 [PATCHv4 00/14] fetch submodules in parallel and a preview on parallel "submodule update" Stefan Beller
2015-09-23  1:45 ` [PATCHv4 01/14] submodule: Send "Fetching submodule <foo>" to standard error Stefan Beller
2015-09-23  1:45 ` [PATCHv4 02/14] xread: poll on non blocking fds Stefan Beller
2015-09-23  1:45 ` [PATCHv4 03/14] xread_nonblock: add functionality to read from fds without blocking Stefan Beller
2015-09-23  1:45 ` [PATCHv4 04/14] strbuf: add strbuf_read_once to read " Stefan Beller
2015-09-23  1:45 ` [PATCHv4 05/14] run-command: factor out return value computation Stefan Beller
2015-09-23  1:45 ` [PATCHv4 06/14] run-command: add an asynchronous parallel child processor Stefan Beller
2015-09-23  6:29   ` Junio C Hamano
2015-09-23 17:53     ` Stefan Beller
2015-09-23 18:04       ` Junio C Hamano
2015-09-23 19:34         ` Junio C Hamano
2015-09-23 19:39           ` Stefan Beller
2015-09-23 19:47             ` Junio C Hamano
2015-09-23  6:47   ` Junio C Hamano
2015-09-23 14:59     ` Junio C Hamano
2015-09-23 17:54       ` Junio C Hamano
2015-09-23 23:41         ` [PATCHv5] Another squash on " Stefan Beller
2015-09-24  2:17           ` Junio C Hamano
2015-09-24 21:13             ` [PATCH 0/2] " Stefan Beller
2015-09-24 21:13               ` [PATCH 2/2] SQUASH for "fetch_populated_submodules: use new parallel job processing" Stefan Beller
2015-09-24 21:13               ` [PATCH 1/2] SQUASH??? Stefan Beller
2015-09-25  0:49                 ` Junio C Hamano
2015-09-25  1:09                   ` Junio C Hamano
2015-09-25 17:52                   ` Stefan Beller
2015-09-25 17:56                     ` Junio C Hamano
2015-09-25  1:08               ` [PATCH 0/2] Another squash on run-command: add an asynchronous parallel child processor Junio C Hamano
2015-09-25 18:56                 ` Stefan Beller
2015-09-25 19:04                   ` Junio C Hamano
2015-09-25 19:19                     ` Stefan Beller
2015-09-25 19:32                   ` Junio C Hamano
2015-09-23  1:45 ` [PATCHv4 07/14] fetch_populated_submodules: use new parallel job processing Stefan Beller
2015-09-23  1:45 ` [PATCHv4 08/14] submodules: allow parallel fetching, add tests and documentation Stefan Beller
2015-09-23  1:45 ` [PATCHv4 09/14] submodule-config: Untangle logic in parse_config Stefan Beller
2015-09-23  1:45 ` [PATCHv4 10/14] submodule config: keep update strategy around Stefan Beller
2015-09-23  1:45 ` [PATCHv4 11/14] git submodule update: cmd_update_recursive Stefan Beller
2015-09-23  1:45 ` [PATCHv4 12/14] git submodule update: cmd_update_clone Stefan Beller
2015-09-23 20:13   ` Junio C Hamano
2015-09-23  1:45 ` [PATCHv4 13/14] git submodule update: cmd_update_fetch Stefan Beller
2015-09-23  1:45 ` [PATCHv4 14/14] Rewrite submodule update in C Stefan Beller

This is an external index of several public inboxes,
see mirroring instructions on how to clone and mirror
all data and code used by this external index.