cockroachdb / cockroach

CockroachDB — the cloud native, distributed SQL database designed for high availability, effortless scale, and control over data placement.
https://www.cockroachlabs.com
Other
30.13k stars 3.81k forks source link

jobs: needs to work without liveness and nodeID #47892

Closed tbg closed 4 years ago

tbg commented 4 years ago

The jobs framework currently uses node liveness to function. For multi-tenancy phase 2, jobs will run on SQL tenant processes which do not have access to the liveness table and which additionally don't have a node ID. This is problematic because jobs uses both to reason about the liveness of jobs via the lease.

As a redeeming feature, phase 2 will have only a single SQL tenant process running on behalf of a given tenant, meaning that job liveness is simple - jobs are always live and jobs that are not running on the tenant should always be resumed.

Past phase 2, we are likely looking at multiple SQL tenant processes running at the same time, and we'll need a discovery/liveness mechanism between them. However, multiple mechanisms will need such a mechanism (for example DistSQL), so it won't have to be developed just for jobs.

tbg commented 4 years ago

This turns out to be relatively urgent, since without jobs not much really works. At the time of writing, the tenant server is only partially what it needs to be (it still gets gossip and a bogus nodeID, etc) and jobs definitely don't work (how could they) plus, we hit this assertion all the time:

diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go
index 55fb55a0ed..52815e5ab0 100644
--- a/pkg/jobs/registry.go
+++ b/pkg/jobs/registry.go
@@ -1119,8 +1119,13 @@ WHERE status IN ($1, $2, $3, $4, $5) ORDER BY created DESC`
                        nodeStatus, ok := nodeStatusMap[payload.Lease.NodeID]
                        if !ok {
                                // This case should never happen.
-                               log.ReportOrPanic(ctx, nil, "job %d: skipping: no liveness record for the job's node %d",
-                                       log.Safe(*id), payload.Lease.NodeID)
+                               //
+                               // TODO(tbg): re-enable this assertion when jobs has been re-worked to function without
+                               // node liveness.
+                               if false {
+                                       log.ReportOrPanic(ctx, nil, "job %d: skipping: no liveness record for the job's node %d",
+                                               log.Safe(*id), payload.Lease.NodeID)
+                               }
                                continue
                        }
                        if nodeStatus.isLive {
ajwerner commented 4 years ago

I was hopeful that in phase 2 we would still be getting a node ID albeit a potentially bogus one. Gossip and node liveness should be relatively straightforward to eliminate but I would very much like an identity for a sql process which can be put into a lease.

ajwerner commented 4 years ago

Bogus in the sense that it may later be reused but unique during the lifetime of that sql pod

tbg commented 4 years ago

I'm fine giving you something to put into that lease (after all that lease is per-tenant and there's never more than one SQL server for that tenant, so any fixed number will do) Which means: you can just hard-code 1 where you need a NodeID?

I want to avoid giving the misguided impression that there will be a NodeID available in the system. I'm fine with jobs pretending there is one as long as the code that uses it is aware that it's fake and documents why that's ok.

tbg commented 4 years ago

(once we have multiple processes running for a tenant - I agree we need a proper substitute for the node id).

ajwerner commented 4 years ago

The vision here as I understand it is to replace the node-liveness epoch based "leases" with heart-beated leases implemented directly on top of kv. Fortunately we have a library that does just this (sqlmigrations/leasemanager).

That library uses an arbitrary string as the ClientID and an arbitrary key to store the encoded lease.

We're going to propose storing the lease in a new column and column family on the existing system.jobs table. This decision is inspired by the decision in the Scheduled Jobs RFC to add a new column to the same table (see https://github.com/cockroachdb/cockroach/blob/master/docs/RFCS/20200414_scheduled_jobs.md#scheduled-job-representation).

The idea is that we'll introduce a new version, which, after indicated, will utilize only the new kv-based leases. Any jobs.Registry which is constructed with this new version active will gladly accept a nil OptionalNodeLiveness (or whatever the right terminology is these days for saying that it really is optional).

Steps:

This step will require a new cockroach version and sqlmigration to perform the upgrade of the table. This will likely make sense as 3-4 commits:

1) Change the structure of the jobs table in sqlbase.

The change in sql/sqlbase/system.go might look something like:

                        {Name: "progress", ID: 5, Type: types.Bytes, Nullable: true},
+                       {Name: "lease", ID: 6, Type: types.Bytes, Nullable: true},
                },
                NextColumnID: 6,
                Families: []ColumnFamilyDescriptor{
@@ -828,8 +832,15 @@ var (
                                ColumnIDs:       []ColumnID{5},
                                DefaultColumnID: 5,
                        },
+                       {
+                               Name:            "lease",
+                               ID:              2,
+                               ColumnNames:     []string{"lease"},
+                               ColumnIDs:       []ColumnID{6},
+                               DefaultColumnID: 6,
+                       },
                },
-               NextFamilyID: 2,
+               NextFamilyID: 3,
                PrimaryIndex: pk("id")

2) Introduce the new version to cockroach_versions.go.

See https://github.com/cockroachdb/cockroach/blob/f4a5cac899802d218c2acd1434792f4259a7eabb/pkg/clusterversion/cockroach_versions.go#L18

3) Add a sqlmigration to do this migration on existing clusters.

4) Add some unit tests to do the actual migration and make sure that it will work with the lease manager. It can be the case that you only instantiate the leasemanager.LeaseManager in unit tests at this point.

It is likely a good idea to add the logic to interact with this column family from KV to ensure that that all works. I imagine that making the lease key for a jobs row will look like:

func makeJobLeaseKey(codec keys.SQLCodec, jobID int64) roachpb.Key {
        prefix := keys.MakeFamilyKey(codec.IndexPrefix(keys.JobsTableID, 1), 2)
    k, err := sqlbase.EncodeTableKey(prefix, (*tree.DInt)(&jobID), encoding.Ascending)
    if err != nil {
        panic(err)
    }
    return k
}

At this point we should ignore the cluster version and just make sure that running jobs acquire both leases.

One point here is that we'll need a new loop to refresh all of the various leases. The epoch-based leases have a loop to just check whether they have failed but doesn't actually have to do anything to prevent the leases from failing. Upon failure to extend a lease, a running job should be canceled.

It is critical that we have code to avoid ever trying to check for the old-style leases if at construction time the new version is active.


Additional steps:

Sequence numbers are handy for debugging and just generally a good idea.

Today's job leases are best effort, a job may have lost its lease but not yet realized it. It only realizes it asynchronously. The fact that these leases are best-effort is something we often forget about. In fact, two concurrently running jobs could, potentially, step on eachother whereby one overwrites status written by a later incarnation. Ideally jobs would ensure that they still hold the lease every time they write to their row. This shouldn't be very hard, it would just require some checking inside of the Update method.

ajwerner commented 4 years ago

Upon some reflection, the original plan of having an individually heartbeated lease ends up conflicting with the user story for jobs which are to be run on the local node and return results. Today we attempt to implement this by initializing the job record with a lease for the current node. Such an approach could also be taken with these new leases whereby we'd initialize the lease to the current node when we write it. The rub, however, is that these new leases will contain timestamps and we don't have any bound on when that user transaction is going to commit. We could work around this, of course, but the question is whether we want to.

Workaround (not the proposal)

Quick aside:

Today's StartableJobs which we run on behalf of the user abuse our transactions and actually use a transaction other than the user transaction. This is a bummer but it is what it is. It does, however, make it easier to imagine that this workaround will Just Work (tm).

Proposal

To work around the above limitation we could make sure to re-write (or write for the first time) the lease for jobs which we intend to run locally prior to committing the transaction. This is highly likely to mean that we'd create a valid lease. Then we'd need the Start method of the StartableJob to interact with the job registry's heartbeat loop.

Alternate proposal

Create a concept of a session. A session is effectively, in many ways, a re-creation of node-liveness that is decoupled from the KV layer. Each sql pod would maintain a heartbeated record indicating its liveness. Given that this session will be used for jobs leases and perhaps alter for things like DistSQL distribution, its liveness properties can be quite relaxed relative to today's nodeliveness. Think minutes rather than seconds.

This session will live in a new system table. It does not need to be tightly coupled to the jobs registry.

There is an open question about what would happen if the session expires and needs to be re-created. My hunch is nothing drastic except that job leases would be lost and jobs which were running under previous leases should be pre-empted and stopped.

We can gain mutual exclusion for updates to job records safely by utilizing reading the session to ensure that it still exists. This is important because today we're afraid to transactionally interact with node liveness so the guarantees we get out of the jobs system are pretty weak.

Concretely this plan would be:


@spaskob this alternative proposal is a bit sad in that it does not build on the work you've done in the last week but I think is both simpler with regards to the jobs package and a better foundation moving forward. Let's weigh the pros and cons and come to a conclusion.

spaskob commented 4 years ago

@ajwerner thanks for the nice summary of our discussion! This is has proven much trickier than I initially thought - I have already discarded 2 prototypes that I played with on your ideas.

Here is my current working approach:

  1. We will use you the alternate proposal above of creating the concept of a session that lives in a separate system table. Let's call it tentatively system.sqlliveness.
  2. Instead of writing the session id in a new field in job Payload we will use a new column named LeasedBy in table systems.jobs. The column will contain the session id of the registry that is currently running the job, a default invalid id for jobs that have not been adopted or NULL for legacy jobs.
  3. The registry will periodically ping its row in system.sqlliveness to keep the session alive, if it discovers that the session sequence number has been bumped, the registry will restart itself, cancelling all its running jobs.
  4. As suggested in 3, other registries can periodically check on sessions of existing jobs and if they have expired will bump their sequence and steal the job by adopting it and writing their own session id in the column LeasedBy.

This describes the new world. Now how about the transition of existing clusters to it?

Cluster upgrade steps

Stage 1. Mixed-version cluster

Old nodes will just work as is. Remember that new style jobs will not be created so there is no danger of old nodes trying to steal new style jobs as none will be created. New nodes will create jobs using the old leasing mechanism and will not populate the new column LeasedBy, ie they will use the old code paths as well. Effectively all jobs will have NULL in the LeasedBy column and the registries will behave as of old.

Stage 2. Cluster is upgraded but not finalized

Same as Stage 1.

Stage 3. Cluster is finalized

Newly created jobs will have column LeasedBy set. The new adoption loop will take care of all newly created jobs.

Legacy jobs that have not finished yet will be adopted by the old loop. As new legacy jobs will not be created eventually we will detect that there are none left and the old adoption loop will cancel itself.

Benefits of using a separate column

ajwerner commented 4 years ago

This SGTM. One thing that isn't clear to me from the description is what will live in that new column. Will it be also an encoded protocol buffer?

Also, as an optimization, you could have the new polling loop only poll if the new version is active.

ajwerner commented 4 years ago

I typed up a summary of the sqlliveness discussions as I understand them in this short RFC: https://github.com/cockroachdb/cockroach/pull/50377/files?short_path=61eb97d#diff-61eb97dcc595a8a1348edc6ecd53c755

tbg commented 4 years ago

Removed the blocker label since - as far as I can tell - we are OK for initial launch with the current code and the work being done now is just to put things in a better place generally. Please ring a bell if that's incorrect @ajwerner

ajwerner commented 4 years ago

So long as there is only one sql pod for a given tenant, a dummy node-liveness can work. I’m not super clear on what happens now when the tenant attempts to access node liveness or its node id.

spaskob commented 4 years ago

@tbg I've submitted https://github.com/cockroachdb/cockroach/pull/51087 which changes job to use a different leasing mechanism. Is there anything left to be done on this issue?

tbg commented 4 years ago

No, thanks for your work!