[gold] Add gitilesfollower

This is a simplified version of gitsync that works directly
with the SQL database and gitiles. The idea here is that
this process will make a mapping of commit_id and git_hash
in the GitCommits table, so when we see those git_hash
during ingestion, we can derive a commit_id and use that
for the actual ordering of data.

This version takes a lesson from Perf by only querying data
from the most recent commit in the DB and the latest on
the tree to make Gold resilient to merged/changed
history (e.g. go/skia-infra-pm-007)

As a result, I've made the commit_id field required for
the GitCommits table.

Suggested review order:
 - sql.go and tables.go
 - gitilesfollower_manual_test.go
 - gittilesfollower_test.go
 - gitilesfollower.go
 - Everything else

Bug: skia:10582, skia:11367
Change-Id: I0b6d3e89c91ff38aeef99f11b02775f644ed819d
Reviewed-on: https://skia-review.googlesource.com/c/buildbot/+/375664
Reviewed-by: Leandro Lovisolo <lovisolo@google.com>
diff --git a/golden/cmd/gitilesfollower/gitilesfollower.go b/golden/cmd/gitilesfollower/gitilesfollower.go
new file mode 100644
index 0000000..40fbd0c
--- /dev/null
+++ b/golden/cmd/gitilesfollower/gitilesfollower.go
@@ -0,0 +1,270 @@
+// The gitilesfollower executable monitors the repo we are tracking using gitiles. It fills in
+// the GitCommits table, specifically making a mapping between a GitHash and CommitID. The CommitID
+// is based on the "index" of the commit (i.e. how many commits since the initial commit).
+//
+// This will be used by all clients that have their tests in the same repo as the code under test.
+// Clients with more complex repo structures, will need to have an alternate way of linking
+// commit_id to git_hash.
+package main
+
+import (
+	"context"
+	"flag"
+	"fmt"
+	"net/http"
+	"strconv"
+	"time"
+
+	"github.com/jackc/pgx/v4"
+	"github.com/jackc/pgx/v4/pgxpool"
+	"go.opencensus.io/trace"
+
+	"go.skia.org/infra/go/auth"
+	"go.skia.org/infra/go/common"
+	"go.skia.org/infra/go/gitiles"
+	"go.skia.org/infra/go/httputils"
+	"go.skia.org/infra/go/skerr"
+	"go.skia.org/infra/go/sklog"
+	"go.skia.org/infra/go/vcsinfo"
+	"go.skia.org/infra/golden/go/config"
+	"go.skia.org/infra/golden/go/sql"
+	"go.skia.org/infra/golden/go/tracing"
+)
+
+const (
+	// Arbitrary number
+	maxSQLConnections = 4
+
+	// The initial commit will be given this commit ID. Subsequent commits will have monotonically
+	// increasing integers as IDs. We pick this number instead of zero in case we need to go
+	// backwards, we can assign a non-negative integer as an id (which won't break the sort order
+	// when turned into a string).
+	initialID = 1_000_000_000
+
+	// If overrideLatestCommit is set on a context, the associated value will be used instead of
+	// querying gitiles (which changes over time). This is used by tests.
+	overrideLatestCommitKey = contextKey("override_latest_commit")
+)
+
+type contextKey string // See advice in https://golang.org/pkg/context/#WithValue
+
+type repoFollowerConfig struct {
+	config.Common
+
+	// InitialCommit that we will use if there are no existing commits in the DB. It will be counted
+	// like a "commit zero", which we actually assign to commit 1 billion in case we need to go back
+	// in time, we can sort our commit_ids without resorting to negative numbers.
+	InitialCommit string `json:"initial_commit"`
+
+	// PollPeriod is how often we should poll the source of truth.
+	PollPeriod config.Duration `json:"poll_period"`
+
+	// Metrics service address (e.g., ':10110')
+	PromPort string `json:"prom_port"`
+
+	// The port to provide a web handler for /healthz
+	ReadyPort string `json:"ready_port"`
+}
+
+func main() {
+	// Command line flags.
+	var (
+		commonInstanceConfig = flag.String("common_instance_config", "", "Path to the json5 file containing the configuration that needs to be the same across all services for a given instance.")
+		thisConfig           = flag.String("config", "", "Path to the json5 file containing the configuration specific to baseline server.")
+		hang                 = flag.Bool("hang", false, "Stop and do nothing after reading the flags. Good for debugging containers.")
+	)
+
+	// Parse the options. So we can configure logging.
+	flag.Parse()
+
+	if *hang {
+		sklog.Info("Hanging")
+		select {}
+	}
+
+	var rfc repoFollowerConfig
+	if err := config.LoadFromJSON5(&rfc, commonInstanceConfig, thisConfig); err != nil {
+		sklog.Fatalf("Reading config: %s", err)
+	}
+	sklog.Infof("Loaded config %#v", rfc)
+
+	// Set up the logging options.
+	logOpts := []common.Opt{
+		common.PrometheusOpt(&rfc.PromPort),
+	}
+
+	common.InitWithMust("gitilesfollower", logOpts...)
+	if err := tracing.Initialize(1); err != nil {
+		sklog.Fatalf("Could not set up tracing: %s", err)
+	}
+
+	ctx := context.Background()
+	db := mustInitSQLDatabase(ctx, rfc)
+
+	ts, err := auth.NewDefaultTokenSource(false, auth.SCOPE_USERINFO_EMAIL, auth.SCOPE_GERRIT)
+	if err != nil {
+		sklog.Fatalf("Problem setting up default token source: %s", err)
+	}
+	client := httputils.DefaultClientConfig().WithTokenSource(ts).Client()
+	gitilesClient := gitiles.NewRepo(rfc.GitRepoURL, client)
+	go pollRepo(ctx, db, gitilesClient, rfc)
+
+	// Wait at least 5 seconds for polling to start before signaling all is well.
+	time.Sleep(5 * time.Second)
+	http.HandleFunc("/healthz", httputils.ReadyHandleFunc)
+	sklog.Fatal(http.ListenAndServe(rfc.ReadyPort, nil))
+}
+
+func mustInitSQLDatabase(ctx context.Context, fcc repoFollowerConfig) *pgxpool.Pool {
+	if fcc.SQLDatabaseName == "" {
+		sklog.Fatalf("Must have SQL Database Information")
+	}
+	url := sql.GetConnectionURL(fcc.SQLConnection, fcc.SQLDatabaseName)
+	conf, err := pgxpool.ParseConfig(url)
+	if err != nil {
+		sklog.Fatalf("error getting postgres config %s: %s", url, err)
+	}
+
+	conf.MaxConns = maxSQLConnections
+	db, err := pgxpool.ConnectConfig(ctx, conf)
+	if err != nil {
+		sklog.Fatalf("error connecting to the database: %s", err)
+	}
+	sklog.Infof("Connected to SQL database %s", fcc.SQLDatabaseName)
+	return db
+}
+
+// pollRepo polls the gitiles repo according to the provided duration for as long as the
+// context remains ok.
+func pollRepo(ctx context.Context, db *pgxpool.Pool, client *gitiles.Repo, rfc repoFollowerConfig) {
+	ct := time.Tick(rfc.PollPeriod.Duration)
+	for {
+		select {
+		case <-ctx.Done():
+			sklog.Errorf("Stopping polling due to context error: %s", ctx.Err())
+			return
+		case <-ct:
+			err := updateCycle(ctx, db, client, rfc)
+			if err != nil {
+				sklog.Errorf("Error on this cycle for talking to %s: %s", rfc.GitRepoURL, rfc)
+			}
+		}
+	}
+}
+
+// GitilesLogger is a subset of the gitiles client library that we need. This allows us to mock
+// it out during tests.
+type GitilesLogger interface {
+	Log(ctx context.Context, logExpr string, opts ...gitiles.LogOption) ([]*vcsinfo.LongCommit, error)
+	LogFirstParent(ctx context.Context, from, to string, opts ...gitiles.LogOption) ([]*vcsinfo.LongCommit, error)
+}
+
+// updateCycle polls the gitiles repo for the latest commit and the database for the previously
+// seen commit. If those are different, it polls gitiles for all commits that happened between
+// those two points and stores them to the DB.
+func updateCycle(ctx context.Context, db *pgxpool.Pool, client GitilesLogger, rfc repoFollowerConfig) error {
+	ctx, span := trace.StartSpan(ctx, "gitilesfollower_updateCycle")
+	defer span.End()
+	latestHash, err := getLatestCommitFromRepo(ctx, client, rfc)
+	if err != nil {
+		return skerr.Wrap(err)
+	}
+
+	previousHash, previousID, err := getPreviousCommitFromDB(ctx, db)
+	if err != nil {
+		return skerr.Wrapf(err, "getting recent commits from DB")
+	}
+
+	if previousHash == latestHash {
+		sklog.Infof("no updates - latest seen commit %s", previousHash)
+		return nil
+	}
+	if previousHash == "" {
+		previousHash = rfc.InitialCommit
+		previousID = initialID
+	}
+
+	sklog.Infof("Getting git history from %s to %s", previousHash, latestHash)
+	commits, err := client.LogFirstParent(ctx, previousHash, latestHash)
+	if err != nil {
+		return skerr.Wrapf(err, "getting backlog of commits from %s..%s", previousHash, latestHash)
+	}
+	// commits is backwards and LogFirstParent does not respect gitiles.LogReverse()
+	sklog.Infof("Got %d commits to store", len(commits))
+	if err := storeCommits(ctx, db, previousID, commits); err != nil {
+		return skerr.Wrapf(err, "storing %d commits to GitCommits table", len(commits))
+	}
+	return nil
+}
+
+// getLatestCommitFromRepo returns the git hash of the latest git commit known on the configured
+// branch. If overrideLatestCommitKey has a value set, that will be used instead.
+func getLatestCommitFromRepo(ctx context.Context, client GitilesLogger, rfc repoFollowerConfig) (string, error) {
+	if hash := ctx.Value(overrideLatestCommitKey); hash != nil {
+		return hash.(string), nil
+	}
+	ctx, span := trace.StartSpan(ctx, "gitilesfollower_getLatestCommitFromRepo")
+	defer span.End()
+	latestCommit, err := client.Log(ctx, rfc.GitRepoBranch, gitiles.LogLimit(1))
+	if err != nil {
+		return "", skerr.Wrapf(err, "getting last commit")
+	}
+	if len(latestCommit) < 1 {
+		return "", skerr.Fmt("No commits returned")
+	}
+	sklog.Debugf("latest commit: %#v", latestCommit[0])
+	return latestCommit[0].Hash, nil
+}
+
+// getPreviousCommitFromDB returns the git_hash and the commit_id of the most recently stored
+// commit. "Most recent" here is defined by the lexicographical order of the commit_id. Of note,
+// commit_id is returned as an integer because subsequent ids will be computed by adding to that
+// integer value.
+//
+// This approach takes a lesson from Perf by only querying data from the most recent commit in the
+// DB and the latest on the tree to make Gold resilient to merged/changed history.
+// (e.g. go/skia-infra-pm-007)
+func getPreviousCommitFromDB(ctx context.Context, db *pgxpool.Pool) (string, int64, error) {
+	ctx, span := trace.StartSpan(ctx, "gitilesfollower_getPreviousCommitFromDB")
+	defer span.End()
+	row := db.QueryRow(ctx, `SELECT git_hash, commit_id FROM GitCommits
+ORDER BY commit_id DESC LIMIT 1`)
+	hash := ""
+	id := ""
+	if err := row.Scan(&hash, &id); err != nil {
+		if err == pgx.ErrNoRows {
+			return "", 0, nil // No data in GitCommits
+		}
+		return "", 0, skerr.Wrap(err)
+	}
+	idInt, err := strconv.ParseInt(id, 10, 64)
+	if err != nil {
+		return "", 0, skerr.Wrapf(err, "It is assumed that the commit ids for this type of repo tracking are ints: %q", id)
+	}
+	return hash, idInt, nil
+}
+
+// storeCommits writes the given commits to the SQL database, assigning them commitIDs in
+// monotonically increasing order. The commits slice is expected to be sorted with the most recent
+// commit first (as is returned by gitiles).
+func storeCommits(ctx context.Context, db *pgxpool.Pool, lastCommitID int64, commits []*vcsinfo.LongCommit) error {
+	ctx, span := trace.StartSpan(ctx, "gitilesfollower_storeCommits")
+	defer span.End()
+	const statement = `UPSERT INTO GitCommits (git_hash, commit_id, commit_time, author_email, subject) VALUES `
+	const valuesPerRow = 5
+	arguments := make([]interface{}, 0, len(commits)*valuesPerRow)
+	commitID := lastCommitID + 1
+	for i := range commits {
+		// commits is in backwards order. This reverses things.
+		c := commits[len(commits)-i-1]
+		cid := fmt.Sprintf("%012d", commitID)
+		arguments = append(arguments, c.Hash, cid, c.Timestamp, c.Author, c.Subject)
+		commitID++
+	}
+	vp := sql.ValuesPlaceholders(valuesPerRow, len(commits))
+	if _, err := db.Exec(ctx, statement+vp, arguments...); err != nil {
+		return skerr.Wrap(err)
+	}
+	return nil
+
+}
diff --git a/golden/cmd/gitilesfollower/gitilesfollower_manual_test.go b/golden/cmd/gitilesfollower/gitilesfollower_manual_test.go
new file mode 100644
index 0000000..e60484e
--- /dev/null
+++ b/golden/cmd/gitilesfollower/gitilesfollower_manual_test.go
@@ -0,0 +1,97 @@
+package main
+
+import (
+	"context"
+	"testing"
+	"time"
+
+	"github.com/jackc/pgx/v4/pgxpool"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/require"
+
+	"go.skia.org/infra/go/gitiles"
+	"go.skia.org/infra/go/httputils"
+	"go.skia.org/infra/go/testutils/unittest"
+	"go.skia.org/infra/golden/go/config"
+	"go.skia.org/infra/golden/go/sql/schema"
+	"go.skia.org/infra/golden/go/sql/sqltest"
+)
+
+func TestUpdateCycle_Load1501CommitsFromGitiles_Success(t *testing.T) {
+	unittest.ManualTest(t)
+
+	rfc := repoFollowerConfig{
+		Common: config.Common{
+			GitRepoBranch: "master",
+			GitRepoURL:    "https://skia.googlesource.com/skia.git",
+		},
+		// Arbitrary commit from 14 Dec 2020
+		InitialCommit: "9b395f55ea0f8f92103d33f1ea8e8217bee8aaea",
+	}
+	// Pretend the test is always running on the morning of 26 Feb 2021, with this
+	// commit being the most recent. This commit is 1502 commits after the InitialCommit.
+	// Since the gitiles API starts after the initial commit, we expect to load 1501 commits.
+	const latestGitHash = "453f143dba3fea76bc777b7d6d933c4017f7e4e8"
+	ctx := overrideLatestGitHash(latestGitHash)
+	db := sqltest.NewCockroachDBForTestsWithProductionSchema(ctx, t)
+
+	gitilesClient := gitiles.NewRepo(rfc.GitRepoURL, httputils.NewTimeoutClient())
+	err := updateCycle(ctx, db, gitilesClient, rfc)
+	require.NoError(t, err)
+
+	row := db.QueryRow(ctx, `SELECT count(*) FROM GitCommits`)
+	n := 0
+	require.NoError(t, row.Scan(&n))
+	assert.Equal(t, 1501, n)
+
+	// Spot check some of the entries
+	gc := getNthGitCommit(ctx, t, db, 0)
+	assert.Equal(t, schema.GitCommitRow{
+		GitHash:     latestGitHash,
+		CommitID:    "001000001501",
+		CommitTime:  time.Date(2021, time.February, 26, 14, 01, 07, 0, time.UTC),
+		AuthorEmail: "Joh...",
+		Subject:     "Improve dead-code elimination check in SPIR-V.",
+	}, gc)
+
+	gc = getNthGitCommit(ctx, t, db, 500)
+	assert.Equal(t, schema.GitCommitRow{
+		GitHash:     "4d76f63e45f107c9e041ef7ae6534b00b4623044",
+		CommitID:    "001000001001",
+		CommitTime:  time.Date(2021, time.February, 3, 22, 50, 28, 0, time.UTC),
+		AuthorEmail: "Bri...",
+		Subject:     "Fix particle bug where uniforms are allocated too late",
+	}, gc)
+
+	gc = getNthGitCommit(ctx, t, db, 1000)
+	assert.Equal(t, schema.GitCommitRow{
+		GitHash:     "9e1cedda632e7fc20147e4e0b4b2f6dc3728283f",
+		CommitID:    "001000000501",
+		CommitTime:  time.Date(2021, time.January, 14, 14, 38, 18, 0, time.UTC),
+		AuthorEmail: "Der...",
+		Subject:     "Add generic uniform setter function to SkRuntimeShaderBuilder",
+	}, gc)
+
+	gc = getNthGitCommit(ctx, t, db, 1500)
+	assert.Equal(t, schema.GitCommitRow{
+		GitHash:     "f79b298b9d0fb57827e43967ee8bb799c37f7135",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2020, time.December, 14, 20, 04, 07, 0, time.UTC),
+		AuthorEmail: "Mic...",
+		Subject:     "Remove UPDATE_DEVICE_CLIP macro",
+	}, gc)
+}
+
+func getNthGitCommit(ctx context.Context, t *testing.T, db *pgxpool.Pool, n int) schema.GitCommitRow {
+	row := db.QueryRow(ctx, `SELECT * FROM GitCommits ORDER BY commit_id DESC LIMIT 1 OFFSET $1`, n)
+	var gc schema.GitCommitRow
+	require.NoError(t, row.Scan(&gc.GitHash, &gc.CommitID, &gc.CommitTime, &gc.AuthorEmail, &gc.Subject))
+	gc.CommitTime = gc.CommitTime.UTC()
+	// We would rather not put real-world names and emails in public test assertions
+	gc.AuthorEmail = gc.AuthorEmail[:3] + "..."
+	return gc
+}
+
+func overrideLatestGitHash(gitHash string) context.Context {
+	return context.WithValue(context.Background(), overrideLatestCommitKey, gitHash)
+}
diff --git a/golden/cmd/gitilesfollower/gitilesfollower_test.go b/golden/cmd/gitilesfollower/gitilesfollower_test.go
new file mode 100644
index 0000000..36d29ce
--- /dev/null
+++ b/golden/cmd/gitilesfollower/gitilesfollower_test.go
@@ -0,0 +1,273 @@
+package main
+
+import (
+	"context"
+	"testing"
+	"time"
+
+	"github.com/jackc/pgx/v4/pgxpool"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/mock"
+	"github.com/stretchr/testify/require"
+
+	"go.skia.org/infra/go/testutils"
+	"go.skia.org/infra/go/testutils/unittest"
+	"go.skia.org/infra/go/vcsinfo"
+	"go.skia.org/infra/golden/cmd/gitilesfollower/mocks"
+	"go.skia.org/infra/golden/go/config"
+	"go.skia.org/infra/golden/go/sql/schema"
+	"go.skia.org/infra/golden/go/sql/sqltest"
+)
+
+func TestUpdateCycle_EmptyDB_UsesInitialCommit(t *testing.T) {
+	unittest.LargeTest(t)
+	ctx := context.Background()
+	db := sqltest.NewCockroachDBForTestsWithProductionSchema(ctx, t)
+
+	mgl := mocks.GitilesLogger{}
+	mgl.On("Log", testutils.AnyContext, "main", mock.Anything).Return([]*vcsinfo.LongCommit{
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash: "4444444444444444444444444444444444444444",
+				// The rest is ignored from Log
+			},
+		},
+	}, nil)
+
+	mgl.On("LogFirstParent", testutils.AnyContext, "1111111111111111111111111111111111111111", "4444444444444444444444444444444444444444").Return([]*vcsinfo.LongCommit{
+		{ // These are returned with the most recent commits first
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash:    "4444444444444444444444444444444444444444",
+				Author:  "author 4",
+				Subject: "subject 4",
+			},
+			Timestamp: time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		},
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash:    "3333333333333333333333333333333333333333",
+				Author:  "author 3",
+				Subject: "subject 3",
+			},
+			Timestamp: time.Date(2021, time.February, 25, 10, 3, 0, 0, time.UTC),
+		},
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash:    "2222222222222222222222222222222222222222",
+				Author:  "author 2",
+				Subject: "subject 2",
+			},
+			Timestamp: time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		},
+		// LogFirstParent excludes the first one mentioned.
+	}, nil)
+
+	rfc := repoFollowerConfig{
+		Common: config.Common{
+			GitRepoBranch: "main",
+		},
+		InitialCommit: "1111111111111111111111111111111111111111",
+	}
+	require.NoError(t, updateCycle(ctx, db, &mgl, rfc))
+
+	actualRows := getAllGitCommits(ctx, t, db)
+	assert.Equal(t, []schema.GitCommitRow{{
+		GitHash:     "4444444444444444444444444444444444444444",
+		CommitID:    "001000000003",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		AuthorEmail: "author 4",
+		Subject:     "subject 4",
+	}, {
+		GitHash:     "3333333333333333333333333333333333333333",
+		CommitID:    "001000000002",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 3, 0, 0, time.UTC),
+		AuthorEmail: "author 3",
+		Subject:     "subject 3",
+	}, {
+		GitHash:     "2222222222222222222222222222222222222222",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		AuthorEmail: "author 2",
+		Subject:     "subject 2",
+	}}, actualRows)
+	// The initial commit is not stored in the DB nor queried, but is implicitly has id
+	// equal to initialID.
+}
+
+func TestUpdateCycle_CommitsInDB_IncrementalUpdate(t *testing.T) {
+	unittest.LargeTest(t)
+	ctx := context.Background()
+	db := sqltest.NewCockroachDBForTestsWithProductionSchema(ctx, t)
+	existingData := schema.Tables{GitCommits: []schema.GitCommitRow{{
+		GitHash:     "4444444444444444444444444444444444444444",
+		CommitID:    "001000000003",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		AuthorEmail: "author 4",
+		Subject:     "subject 4",
+	}, {
+		GitHash:     "3333333333333333333333333333333333333333",
+		CommitID:    "001000000002",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 3, 0, 0, time.UTC),
+		AuthorEmail: "author 3",
+		Subject:     "subject 3",
+	}, {
+		GitHash:     "2222222222222222222222222222222222222222",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		AuthorEmail: "author 2",
+		Subject:     "subject 2",
+	}}}
+	require.NoError(t, sqltest.BulkInsertDataTables(ctx, db, existingData))
+
+	mgl := mocks.GitilesLogger{}
+	mgl.On("Log", testutils.AnyContext, "main", mock.Anything).Return([]*vcsinfo.LongCommit{
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash: "6666666666666666666666666666666666666666",
+				// The rest is ignored from Log
+			},
+		},
+	}, nil)
+
+	mgl.On("LogFirstParent", testutils.AnyContext, "4444444444444444444444444444444444444444", "6666666666666666666666666666666666666666").Return([]*vcsinfo.LongCommit{
+		{ // These are returned with the most recent commits first
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash:    "6666666666666666666666666666666666666666",
+				Author:  "author 6",
+				Subject: "subject 6",
+			},
+			Timestamp: time.Date(2021, time.February, 25, 10, 6, 0, 0, time.UTC),
+		},
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash:    "5555555555555555555555555555555555555555",
+				Author:  "author 5",
+				Subject: "subject 5",
+			},
+			Timestamp: time.Date(2021, time.February, 25, 10, 5, 0, 0, time.UTC),
+		},
+		// LogFirstParent excludes the first one mentioned.
+	}, nil)
+
+	rfc := repoFollowerConfig{
+		Common: config.Common{
+			GitRepoBranch: "main",
+		},
+		InitialCommit: "1111111111111111111111111111111111111111", // we expect this to not be used
+	}
+	require.NoError(t, updateCycle(ctx, db, &mgl, rfc))
+
+	actualRows := getAllGitCommits(ctx, t, db)
+	assert.Equal(t, []schema.GitCommitRow{{
+		GitHash:     "6666666666666666666666666666666666666666",
+		CommitID:    "001000000005",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 6, 0, 0, time.UTC),
+		AuthorEmail: "author 6",
+		Subject:     "subject 6",
+	}, {
+		GitHash:     "5555555555555555555555555555555555555555",
+		CommitID:    "001000000004",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 5, 0, 0, time.UTC),
+		AuthorEmail: "author 5",
+		Subject:     "subject 5",
+	}, {
+		GitHash:     "4444444444444444444444444444444444444444",
+		CommitID:    "001000000003",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		AuthorEmail: "author 4",
+		Subject:     "subject 4",
+	}, {
+		GitHash:     "3333333333333333333333333333333333333333",
+		CommitID:    "001000000002",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 3, 0, 0, time.UTC),
+		AuthorEmail: "author 3",
+		Subject:     "subject 3",
+	}, {
+		GitHash:     "2222222222222222222222222222222222222222",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		AuthorEmail: "author 2",
+		Subject:     "subject 2",
+	}}, actualRows)
+}
+
+func TestUpdateCycle_NoNewCommits_NothingChanges(t *testing.T) {
+	unittest.LargeTest(t)
+	ctx := context.Background()
+	db := sqltest.NewCockroachDBForTestsWithProductionSchema(ctx, t)
+	existingData := schema.Tables{GitCommits: []schema.GitCommitRow{{
+		GitHash:     "4444444444444444444444444444444444444444",
+		CommitID:    "001000000003",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		AuthorEmail: "author 4",
+		Subject:     "subject 4",
+	}, {
+		GitHash:  "3333333333333333333333333333333333333333",
+		CommitID: "001000000002",
+		// Notice this commit comes the latest temporally, but commit_id is what should be use
+		// to determine recency.
+		CommitTime:  time.Date(2025, time.December, 25, 10, 3, 0, 0, time.UTC),
+		AuthorEmail: "author 3",
+		Subject:     "subject 3",
+	}, {
+		GitHash:     "2222222222222222222222222222222222222222",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		AuthorEmail: "author 2",
+		Subject:     "subject 2",
+	}}}
+	require.NoError(t, sqltest.BulkInsertDataTables(ctx, db, existingData))
+
+	mgl := mocks.GitilesLogger{}
+	mgl.On("Log", testutils.AnyContext, "main", mock.Anything).Return([]*vcsinfo.LongCommit{
+		{
+			ShortCommit: &vcsinfo.ShortCommit{
+				Hash: "4444444444444444444444444444444444444444",
+				// The rest is ignored from Log
+			},
+		},
+	}, nil)
+
+	rfc := repoFollowerConfig{
+		Common: config.Common{
+			GitRepoBranch: "main",
+		},
+		InitialCommit: "1111111111111111111111111111111111111111", // we expect this to not be used
+	}
+	require.NoError(t, updateCycle(ctx, db, &mgl, rfc))
+
+	actualRows := getAllGitCommits(ctx, t, db)
+	assert.Equal(t, []schema.GitCommitRow{{
+		GitHash:     "4444444444444444444444444444444444444444",
+		CommitID:    "001000000003",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 4, 0, 0, time.UTC),
+		AuthorEmail: "author 4",
+		Subject:     "subject 4",
+	}, {
+		GitHash:     "3333333333333333333333333333333333333333",
+		CommitID:    "001000000002",
+		CommitTime:  time.Date(2025, time.December, 25, 10, 3, 0, 0, time.UTC),
+		AuthorEmail: "author 3",
+		Subject:     "subject 3",
+	}, {
+		GitHash:     "2222222222222222222222222222222222222222",
+		CommitID:    "001000000001",
+		CommitTime:  time.Date(2021, time.February, 25, 10, 2, 0, 0, time.UTC),
+		AuthorEmail: "author 2",
+		Subject:     "subject 2",
+	}}, actualRows)
+}
+
+func getAllGitCommits(ctx context.Context, t *testing.T, db *pgxpool.Pool) []schema.GitCommitRow {
+	rows, err := db.Query(ctx, `SELECT * FROM GitCommits ORDER BY commit_id DESC`)
+	require.NoError(t, err)
+	defer rows.Close()
+	var rv []schema.GitCommitRow
+	for rows.Next() {
+		var gc schema.GitCommitRow
+		require.NoError(t, rows.Scan(&gc.GitHash, &gc.CommitID, &gc.CommitTime, &gc.AuthorEmail, &gc.Subject))
+		gc.CommitTime = gc.CommitTime.UTC()
+		rv = append(rv, gc)
+	}
+	return rv
+}
diff --git a/golden/cmd/gitilesfollower/mocks/GitilesLogger.go b/golden/cmd/gitilesfollower/mocks/GitilesLogger.go
new file mode 100644
index 0000000..90c5f7e
--- /dev/null
+++ b/golden/cmd/gitilesfollower/mocks/GitilesLogger.go
@@ -0,0 +1,77 @@
+// Code generated by mockery v2.4.0. DO NOT EDIT.
+
+package mocks
+
+import (
+	context "context"
+
+	mock "github.com/stretchr/testify/mock"
+	gitiles "go.skia.org/infra/go/gitiles"
+
+	vcsinfo "go.skia.org/infra/go/vcsinfo"
+)
+
+// GitilesLogger is an autogenerated mock type for the GitilesLogger type
+type GitilesLogger struct {
+	mock.Mock
+}
+
+// Log provides a mock function with given fields: ctx, logExpr, opts
+func (_m *GitilesLogger) Log(ctx context.Context, logExpr string, opts ...gitiles.LogOption) ([]*vcsinfo.LongCommit, error) {
+	_va := make([]interface{}, len(opts))
+	for _i := range opts {
+		_va[_i] = opts[_i]
+	}
+	var _ca []interface{}
+	_ca = append(_ca, ctx, logExpr)
+	_ca = append(_ca, _va...)
+	ret := _m.Called(_ca...)
+
+	var r0 []*vcsinfo.LongCommit
+	if rf, ok := ret.Get(0).(func(context.Context, string, ...gitiles.LogOption) []*vcsinfo.LongCommit); ok {
+		r0 = rf(ctx, logExpr, opts...)
+	} else {
+		if ret.Get(0) != nil {
+			r0 = ret.Get(0).([]*vcsinfo.LongCommit)
+		}
+	}
+
+	var r1 error
+	if rf, ok := ret.Get(1).(func(context.Context, string, ...gitiles.LogOption) error); ok {
+		r1 = rf(ctx, logExpr, opts...)
+	} else {
+		r1 = ret.Error(1)
+	}
+
+	return r0, r1
+}
+
+// LogFirstParent provides a mock function with given fields: ctx, from, to, opts
+func (_m *GitilesLogger) LogFirstParent(ctx context.Context, from string, to string, opts ...gitiles.LogOption) ([]*vcsinfo.LongCommit, error) {
+	_va := make([]interface{}, len(opts))
+	for _i := range opts {
+		_va[_i] = opts[_i]
+	}
+	var _ca []interface{}
+	_ca = append(_ca, ctx, from, to)
+	_ca = append(_ca, _va...)
+	ret := _m.Called(_ca...)
+
+	var r0 []*vcsinfo.LongCommit
+	if rf, ok := ret.Get(0).(func(context.Context, string, string, ...gitiles.LogOption) []*vcsinfo.LongCommit); ok {
+		r0 = rf(ctx, from, to, opts...)
+	} else {
+		if ret.Get(0) != nil {
+			r0 = ret.Get(0).([]*vcsinfo.LongCommit)
+		}
+	}
+
+	var r1 error
+	if rf, ok := ret.Get(1).(func(context.Context, string, string, ...gitiles.LogOption) error); ok {
+		r1 = rf(ctx, from, to, opts...)
+	} else {
+		r1 = ret.Error(1)
+	}
+
+	return r0, r1
+}
diff --git a/golden/cmd/gitilesfollower/mocks/generate.go b/golden/cmd/gitilesfollower/mocks/generate.go
new file mode 100644
index 0000000..23b25a3
--- /dev/null
+++ b/golden/cmd/gitilesfollower/mocks/generate.go
@@ -0,0 +1,3 @@
+package mocks
+
+//go:generate mockery --name GitilesLogger --dir ../ --output .
diff --git a/golden/go/diff/worker/worker.go b/golden/go/diff/worker/worker.go
index 3543e69..2b85073 100644
--- a/golden/go/diff/worker/worker.go
+++ b/golden/go/diff/worker/worker.go
@@ -55,7 +55,7 @@
 	badImageCooldown = time.Minute
 )
 
-type contextKey string
+type contextKey string // See advice in https://golang.org/pkg/context/#WithValue
 
 // NowSource is an abstraction around a clock.
 type NowSource interface {
diff --git a/golden/go/sql/databuilder/databuilder.go b/golden/go/sql/databuilder/databuilder.go
index 39e885b..0783b3f 100644
--- a/golden/go/sql/databuilder/databuilder.go
+++ b/golden/go/sql/databuilder/databuilder.go
@@ -781,7 +781,7 @@
 	})
 	b.gitRows = append(b.gitRows, schema.GitCommitRow{
 		GitHash:     gitHash,
-		CommitID:    &commitID,
+		CommitID:    commitID,
 		CommitTime:  ct,
 		AuthorEmail: author,
 		Subject:     subject,
@@ -796,7 +796,7 @@
 
 // Insert adds a commit with the given data. It panics if the commitTime is not formatted to
 // RFC3339 or if the gitHash is invalid.
-func (b *GitCommitBuilder) Insert(gitHash string, author, subject, commitTime string) *GitCommitBuilder {
+func (b *GitCommitBuilder) Insert(commitID, gitHash, author, subject, commitTime string) *GitCommitBuilder {
 	if len(gitHash) != 40 {
 		panic("invalid git hash length; must be 40 chars")
 	}
@@ -806,6 +806,7 @@
 	}
 	b.gitRows = append(b.gitRows, schema.GitCommitRow{
 		GitHash:     gitHash,
+		CommitID:    schema.CommitID(commitID),
 		CommitTime:  ct,
 		AuthorEmail: author,
 		Subject:     subject,
diff --git a/golden/go/sql/databuilder/databuilder_test.go b/golden/go/sql/databuilder/databuilder_test.go
index 0a2a4cf..e6795d2 100644
--- a/golden/go/sql/databuilder/databuilder_test.go
+++ b/golden/go/sql/databuilder/databuilder_test.go
@@ -34,7 +34,7 @@
 		Insert("003", "author_three", "subject_three", "2020-12-07T18:00:00Z").
 		Insert("004", "author_four", "subject_four", "2020-12-08T19:00:00Z")
 	b.CommitsWithNoData().
-		Insert("5555555555555555555555555555555555555555", "author_five", "no data yet", "2020-12-08T20:00:00Z")
+		Insert("005", "5555555555555555555555555555555555555555", "author_five", "no data yet", "2020-12-08T20:00:00Z")
 	b.SetDigests(map[rune]types.Digest{
 		// by convention, upper case are positively triaged, lowercase
 		// are untriaged, numbers are negative, symbols are special.
@@ -172,31 +172,31 @@
 	}}, tables.CommitsWithData)
 	assert.Equal(t, []schema.GitCommitRow{{
 		GitHash:     gitHash("001"),
-		CommitID:    cID("001"),
+		CommitID:    "001",
 		CommitTime:  time.Date(2020, time.December, 5, 16, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_one",
 		Subject:     "subject_one",
 	}, {
 		GitHash:     gitHash("002"),
-		CommitID:    cID("002"),
+		CommitID:    "002",
 		CommitTime:  time.Date(2020, time.December, 6, 17, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_two",
 		Subject:     "subject_two",
 	}, {
 		GitHash:     gitHash("003"),
-		CommitID:    cID("003"),
+		CommitID:    "003",
 		CommitTime:  time.Date(2020, time.December, 7, 18, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_three",
 		Subject:     "subject_three",
 	}, {
 		GitHash:     gitHash("004"),
-		CommitID:    cID("004"),
+		CommitID:    "004",
 		CommitTime:  time.Date(2020, time.December, 8, 19, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_four",
 		Subject:     "subject_four",
 	}, {
 		GitHash:     "5555555555555555555555555555555555555555",
-		CommitID:    nil,
+		CommitID:    "005",
 		CommitTime:  time.Date(2020, time.December, 8, 20, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_five",
 		Subject:     "no data yet",
@@ -476,10 +476,6 @@
 	}}, tables.IgnoreRules)
 }
 
-func cID(s schema.CommitID) *schema.CommitID {
-	return &s
-}
-
 func TestBuild_CalledWithChangelistData_ProducesCorrectData(t *testing.T) {
 	unittest.SmallTest(t)
 
@@ -906,8 +902,8 @@
 		Insert("2000", "author_2k", "subject_2k", "2022-02-02T02:02:00Z")
 
 	b.CommitsWithNoData().
-		Insert("4444444444444444444444444444444444444444", "somebody", "no data 1900", "2021-02-03T04:05:06Z").
-		Insert("3333333333333333333333333333333333333333", "somebody", "no data 1850", "2021-02-03T04:05:00Z")
+		Insert("1900", "4444444444444444444444444444444444444444", "somebody", "no data 1900", "2021-02-03T04:05:06Z").
+		Insert("1850", "3333333333333333333333333333333333333333", "somebody", "no data 1850", "2021-02-03T04:05:00Z")
 
 	tables := b.Build()
 	assert.Equal(t, []schema.CommitWithDataRow{{
@@ -925,37 +921,37 @@
 	}}, tables.CommitsWithData)
 	assert.Equal(t, []schema.GitCommitRow{{
 		GitHash:     gitHash("0098"),
-		CommitID:    cID("0098"),
+		CommitID:    "0098",
 		CommitTime:  time.Date(2020, time.December, 5, 14, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_two",
 		Subject:     "subject_98",
 	}, {
 		GitHash:     gitHash("0099"),
-		CommitID:    cID("0099"),
+		CommitID:    "0099",
 		CommitTime:  time.Date(2020, time.December, 5, 15, 0, 0, 0, time.UTC),
 		AuthorEmail: "author_one",
 		Subject:     "subject_99",
 	}, {
 		GitHash:     gitHash("0100"),
-		CommitID:    cID("0100"),
+		CommitID:    "0100",
 		CommitTime:  time.Date(2021, time.January, 1, 1, 1, 0, 0, time.UTC),
 		AuthorEmail: "author_100",
 		Subject:     "subject_100",
 	}, {
 		GitHash:     "3333333333333333333333333333333333333333",
-		CommitID:    nil,
+		CommitID:    "1850",
 		CommitTime:  time.Date(2021, time.February, 3, 4, 5, 0, 0, time.UTC),
 		AuthorEmail: "somebody",
 		Subject:     "no data 1850",
 	}, {
 		GitHash:     "4444444444444444444444444444444444444444",
-		CommitID:    nil,
+		CommitID:    "1900",
 		CommitTime:  time.Date(2021, time.February, 3, 4, 5, 6, 0, time.UTC),
 		AuthorEmail: "somebody",
 		Subject:     "no data 1900",
 	}, {
 		GitHash:     gitHash("2000"),
-		CommitID:    cID("2000"),
+		CommitID:    "2000",
 		CommitTime:  time.Date(2022, time.February, 2, 2, 2, 0, 0, time.UTC),
 		AuthorEmail: "author_2k",
 		Subject:     "subject_2k",
diff --git a/golden/go/sql/datakitchensink/kitchensink.go b/golden/go/sql/datakitchensink/kitchensink.go
index 9a19af3..3aaaee4 100644
--- a/golden/go/sql/datakitchensink/kitchensink.go
+++ b/golden/go/sql/datakitchensink/kitchensink.go
@@ -30,9 +30,9 @@
 		Insert("0000000110", UserTwo, "commit 110", "2020-12-11T00:00:00Z")
 
 	b.CommitsWithNoData().
-		Insert("0103010301030103010301030103010301030103", UserFour, "no data 103", "2020-12-06T01:00:00Z").
-		Insert("0104010401040104010401040104010401040104", UserFour, "no data 104", "2020-12-06T02:00:00Z").
-		Insert("0105010501050105010501050105010501050105", UserFour, "no data 105", "2020-12-06T03:00:00Z")
+		Insert("0000000103", "0103010301030103010301030103010301030103", UserFour, "no data 103", "2020-12-06T01:00:00Z").
+		Insert("0000000104", "0104010401040104010401040104010401040104", UserFour, "no data 104", "2020-12-06T02:00:00Z").
+		Insert("0000000105", "0105010501050105010501050105010501050105", UserFour, "no data 105", "2020-12-06T03:00:00Z")
 
 	b.SetDigests(map[rune]types.Digest{
 		// by convention, upper case are positively triaged, lowercase
diff --git a/golden/go/sql/datakitchensink/kitchensink_test.go b/golden/go/sql/datakitchensink/kitchensink_test.go
index 7683404..c7badeb 100644
--- a/golden/go/sql/datakitchensink/kitchensink_test.go
+++ b/golden/go/sql/datakitchensink/kitchensink_test.go
@@ -77,11 +77,6 @@
 	assert.NoError(t, row.Scan(&count))
 	assert.Equal(t, 13, count)
 
-	row = db.QueryRow(ctx, "SELECT count(*) from GitCommits WHERE commit_id IS NULL")
-	count = 0
-	assert.NoError(t, row.Scan(&count))
-	assert.Equal(t, 3, count)
-
 	row = db.QueryRow(ctx, "SELECT count(*) from CommitsWithData")
 	count = 0
 	assert.NoError(t, row.Scan(&count))
diff --git a/golden/go/sql/schema/sql.go b/golden/go/sql/schema/sql.go
index d170558..95f1f92 100644
--- a/golden/go/sql/schema/sql.go
+++ b/golden/go/sql/schema/sql.go
@@ -52,7 +52,7 @@
 );
 CREATE TABLE IF NOT EXISTS GitCommits (
   git_hash STRING PRIMARY KEY,
-  commit_id STRING,
+  commit_id STRING NOT NULL,
   commit_time TIMESTAMP WITH TIME ZONE NOT NULL,
   author_email STRING NOT NULL,
   subject STRING NOT NULL,
diff --git a/golden/go/sql/schema/tables.go b/golden/go/sql/schema/tables.go
index 071a41d..3b8607a 100644
--- a/golden/go/sql/schema/tables.go
+++ b/golden/go/sql/schema/tables.go
@@ -167,9 +167,8 @@
 type GitCommitRow struct {
 	// GitHash is the git hash of the commit.
 	GitHash string `sql:"git_hash STRING PRIMARY KEY"`
-	// CommitID is a potentially arbitrary string. If non-null, it is a foreign key in the
-	// CommitsWithData table.
-	CommitID *CommitID `sql:"commit_id STRING"`
+	// CommitID is a potentially arbitrary string. It is a foreign key in the CommitsWithData table.
+	CommitID CommitID `sql:"commit_id STRING NOT NULL"`
 	// CommitTime is the timestamp associated with the commit.
 	CommitTime time.Time `sql:"commit_time TIMESTAMP WITH TIME ZONE NOT NULL"`
 	// AuthorEmail is the email address associated with the author.