From 8fa9b099254b9c3df68c6a8dee998e6e95bd0d3c Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Tue, 10 Dec 2024 11:21:47 -0600 Subject: [PATCH 1/9] mainboilerplate: optional private diagnostics port When `Port` is set, serve diagnostics (on http.DefaultServeMux) over the indicated port. When `Private` is set, don't serve diagnostics over the main service port. --- cmd/gazette/main.go | 8 ++++++++ mainboilerplate/diagnostics.go | 7 ++++++- mainboilerplate/runconsumer/run_consumer.go | 8 ++++++++ server/server.go | 2 +- 4 files changed, 23 insertions(+), 2 deletions(-) diff --git a/cmd/gazette/main.go b/cmd/gazette/main.go index 2967abda..96844ac1 100644 --- a/cmd/gazette/main.go +++ b/cmd/gazette/main.go @@ -3,6 +3,7 @@ package main import ( "context" "crypto/tls" + "net/http" "os" "os/signal" "syscall" @@ -90,6 +91,13 @@ func (cmdServe) Execute(args []string) error { srv, err := server.New("", Config.Broker.Host, Config.Broker.Port, serverTLS, peerTLS, Config.Broker.MaxGRPCRecvSize, nil) mbp.Must(err, "building Server instance") + if !Config.Diagnostics.Private { + // Expose diagnostics over the main service port. + srv.HTTPMux = http.DefaultServeMux + } else if Config.Diagnostics.Port == "" { + log.Warn("diagnostics are not served over the public port, and a private port is not configured") + } + // If a file:// root was provided, ensure it exists and apply it. if Config.Broker.FileRoot != "" { _, err = os.Stat(Config.Broker.FileRoot) diff --git a/mainboilerplate/diagnostics.go b/mainboilerplate/diagnostics.go index 99f066de..c8c48d52 100644 --- a/mainboilerplate/diagnostics.go +++ b/mainboilerplate/diagnostics.go @@ -15,7 +15,8 @@ import ( // DiagnosticsConfig configures pull-based application metrics, debugging and diagnostics. type DiagnosticsConfig struct { - // Nothing to see here (yet). + Port string `long:"port" env:"PORT" description:"Port for diagnostics"` + Private bool `long:"private" env:"PRIVATE" description:"When true, don't serve diagnostics over the main service port"` } // InitDiagnosticsAndRecover enables serving of metrics and debugging services @@ -40,6 +41,10 @@ func InitDiagnosticsAndRecover(cfg DiagnosticsConfig) func() { // Serve Prometheus metrics at /debug/metrics. http.Handle("/debug/metrics", promhttp.Handler()) + if cfg.Port != "" { + go http.ListenAndServe(":"+cfg.Port, http.DefaultServeMux) + } + return func() { if r := recover(); r != nil { writeExitMessage(r) diff --git a/mainboilerplate/runconsumer/run_consumer.go b/mainboilerplate/runconsumer/run_consumer.go index 7179d052..a4d1f23d 100644 --- a/mainboilerplate/runconsumer/run_consumer.go +++ b/mainboilerplate/runconsumer/run_consumer.go @@ -9,6 +9,7 @@ import ( "crypto/tls" "fmt" "net" + "net/http" "os" "os/signal" "syscall" @@ -167,6 +168,13 @@ func (sc Cmd) Execute(args []string) error { ) mbp.Must(err, "building Server instance") + if !bc.Diagnostics.Private { + // Expose diagnostics over the main service port. + srv.HTTPMux = http.DefaultServeMux + } else if bc.Diagnostics.Port == "" { + log.Warn("diagnostics are not served over the public port, and a private port is not configured") + } + if bc.Broker.Cache.Size <= 0 { log.Warn("--broker.cache.size is disabled; consider setting > 0") } diff --git a/server/server.go b/server/server.go index 293982d4..ceb0d82d 100644 --- a/server/server.go +++ b/server/server.go @@ -101,7 +101,7 @@ func New( var srv = &Server{ endpoint: pb.Endpoint(endpoint), - HTTPMux: http.DefaultServeMux, + HTTPMux: http.NewServeMux(), GRPCServer: grpc.NewServer( grpc.StreamInterceptor(grpc_prometheus.StreamServerInterceptor), grpc.UnaryInterceptor(grpc_prometheus.UnaryServerInterceptor), From ab0d6744db7c404ee797b119c646b29987b5811c Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sun, 15 Dec 2024 12:36:19 -0600 Subject: [PATCH 2/9] broker: fragment persister should deallocate empty queues Truncating slices but holding their memory means that the garbage collector is unable to free the backing files of persisted spools which happen to be resident in the queue. --- broker/fragment/persister.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/broker/fragment/persister.go b/broker/fragment/persister.go index bb76d1b3..a853720b 100644 --- a/broker/fragment/persister.go +++ b/broker/fragment/persister.go @@ -82,7 +82,7 @@ func (p *Persister) Serve() { // Rotate queues. p.mu.Lock() - p.qA, p.qB, p.qC = p.qB, p.qC, p.qA[:0] + p.qA, p.qB, p.qC = p.qB, p.qC, nil if exiting && len(p.qA) == 0 && len(p.qB) == 0 { done = true From e003a7323c1efeee0d73eb4c5bd45db6ce4a12a8 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sun, 15 Dec 2024 12:37:36 -0600 Subject: [PATCH 3/9] broker: fix a potential leaked context --- broker/replica.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/broker/replica.go b/broker/replica.go index cc61da30..006944fd 100644 --- a/broker/replica.go +++ b/broker/replica.go @@ -64,9 +64,9 @@ func fragmentRefreshDaemon(ks *keyspace.KeySpace, r *replica) { for { select { - case _ = <-r.ctx.Done(): + case <-r.ctx.Done(): return - case _ = <-timer.C: + case <-timer.C: } var spec *pb.JournalSpec @@ -117,7 +117,7 @@ func pulseDaemon(svc *Service, r *replica) { invalidateCh = nil } - var ctx, _ = context.WithTimeout(r.ctx, healthCheckInterval) + var ctx, cancel = context.WithTimeout(r.ctx, healthCheckInterval) var fsm = appendFSM{ svc: svc, ctx: ctx, @@ -161,6 +161,7 @@ func pulseDaemon(svc *Service, r *replica) { if fsm.resolved != nil { invalidateCh = fsm.resolved.invalidateCh } + cancel() } } From e6b4cc5477b08b4f99f879ff12f916bd4006cbd5 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 9 Jan 2025 13:22:17 -0600 Subject: [PATCH 4/9] broker: implement journal suspension Introduce a journal suspension mechanism which is incorporated into the Append RPC state machine, and stored as a `suspend` field within each JournalSpec. Suspended journals consumer fewer or no assignment slots, reducing broker and Etcd resources required for journals which are not actively serving appends. When running with the `--auto-suspend` flag, brokers will automatically suspend journals which are fully flushed to the fragment index. --- broker/append_fsm.go | 163 +++- broker/append_fsm_test.go | 74 ++ broker/client/appender.go | 25 +- broker/client/reader.go | 4 + broker/client/retry_reader.go | 2 + broker/fragment/index.go | 7 +- broker/fragment/index_test.go | 6 +- broker/protocol/journal_spec_extensions.go | 84 +- .../protocol/journal_spec_extensions_test.go | 19 +- broker/protocol/protocol.pb.go | 775 +++++++++++++----- broker/protocol/protocol.proto | 65 ++ broker/protocol/rpc_extensions.go | 14 + broker/protocol/rpc_extensions_test.go | 5 + broker/replica.go | 80 +- broker/replicate_api_test.go | 4 +- broker/resolver.go | 16 +- cmd/gazctl/gazctlcmd/journals_fragments.go | 6 + cmd/gazctl/gazctlcmd/journals_list.go | 10 +- cmd/gazctl/gazctlcmd/journals_read.go | 33 +- cmd/gazctl/gazctlcmd/journals_suspend.go | 92 +++ cmd/gazette/main.go | 6 +- 21 files changed, 1223 insertions(+), 267 deletions(-) create mode 100644 cmd/gazctl/gazctlcmd/journals_suspend.go diff --git a/broker/append_fsm.go b/broker/append_fsm.go index 91a43e78..f8b49023 100644 --- a/broker/append_fsm.go +++ b/broker/append_fsm.go @@ -24,19 +24,21 @@ type appendFSM struct { claims pb.Claims req pb.AppendRequest - resolved *resolution // Current journal resolution. - pln *pipeline // Current replication pipeline. - plnReturnCh chan<- *pipeline // If |pln| is owned, channel to which it must be returned. Else nil. - readThroughRev int64 // Etcd revision we must read through to proceed. - rollToOffset int64 // Journal write offset we must synchronize on to proceed. - registers pb.LabelSet // Effective journal registers. - clientCommit bool // Did we see a commit chunk from the client? - clientFragment *pb.Fragment // Journal Fragment holding the client's content. - clientSummer hash.Hash // Summer over the client's content. - clientTotalChunks int64 // Total number of append chunks. - clientDelayedChunks int64 // Number of flow-controlled chunks. - state appendState // Current FSM state. - err error // Error encountered during FSM execution. + resolved *resolution // Current journal resolution. + pln *pipeline // Current replication pipeline. + plnHeader bool // If true, then send opening pipeline header. + plnReturnCh chan<- *pipeline // If |pln| is owned, channel to which it must be returned. Else nil. + readThroughRev int64 // Etcd revision we must read through to proceed. + rollToOffset int64 // Journal write offset we must synchronize on to proceed. + registers pb.LabelSet // Effective journal registers. + nextSuspend *pb.JournalSpec_Suspend // Journal Suspend to apply. + clientCommit bool // Did we see a commit chunk from the client? + clientFragment *pb.Fragment // Journal Fragment holding the client's content. + clientSummer hash.Hash // Summer over the client's content. + clientTotalChunks int64 // Total number of append chunks. + clientDelayedChunks int64 // Number of flow-controlled chunks. + state appendState // Current FSM state. + err error // Error encountered during FSM execution. } type appendState int8 @@ -50,6 +52,7 @@ const ( stateUpdateAssignments appendState = iota stateAwaitDesiredReplicas appendState = iota stateValidatePreconditions appendState = iota + stateUpdateSuspend appendState = iota stateStreamContent appendState = iota // Semi-terminal state (requires more input). stateReadAcknowledgements appendState = iota stateError appendState = iota // Terminal state. @@ -106,6 +109,8 @@ func (b *appendFSM) runTo(state appendState) bool { b.onAwaitDesiredReplicas() case stateValidatePreconditions: b.onValidatePreconditions() + case stateUpdateSuspend: + b.onUpdateSuspend() case stateError, stateProxy, stateFinished, stateStreamContent: return false default: @@ -139,6 +144,12 @@ func (b *appendFSM) onResolve() { if b.resolved, b.err = b.svc.resolver.resolve(b.ctx, b.claims, b.req.Journal, opts); b.err != nil { b.state = stateError b.err = errors.WithMessage(b.err, "resolve") + } else if b.resolved.status == pb.Status_SUSPENDED && b.req.Suspend == pb.AppendRequest_SUSPEND_RESUME { + b.nextSuspend = &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_NONE, + Offset: b.resolved.journalSpec.Suspend.GetOffset(), + } + b.state = stateUpdateSuspend } else if b.resolved.status != pb.Status_OK { b.state = stateError } else if b.resolved.ProcessId != b.resolved.localID { @@ -246,6 +257,7 @@ func (b *appendFSM) onStartPipeline() { // Build a pipeline around `spool`. b.pln = newPipeline(plnCtx, b.resolved.Header, spool, b.resolved.replica.spoolCh, b.svc.jc) + b.plnHeader = true b.state = stateSendPipelineSync } @@ -262,12 +274,12 @@ func (b *appendFSM) onSendPipelineSync() { Registers: &b.registers, Acknowledge: true, } - // Iff |rollToOffset| is zero then this is our first sync of this pipeline, - // and we must attach a Header. + // Iff `plnHeader` then this is our first sync of this pipeline, and we must attach a Header. // TODO: Remove DeprecatedJournal, which is sent for compatibility with last release. - if b.rollToOffset == 0 { + if b.plnHeader { req.Header = &b.pln.Header req.DeprecatedJournal = b.pln.spool.Journal + b.plnHeader = false } b.pln.scatter(req) @@ -423,31 +435,108 @@ func (b *appendFSM) onValidatePreconditions() { // reflected in our index, if a commit wasn't accepted by all peers. // Such writes are reported as failed to the client and are retried // (this failure mode is what makes journals at-least-once). - var indexMin, indexMax = b.resolved.replica.index.OffsetRange() + var indexMin, indexMax, indexDirty = b.resolved.replica.index.OffsetRange() + var suspend = b.resolved.journalSpec.Suspend var maxOffset = b.pln.spool.End if indexMax > maxOffset { maxOffset = indexMax } + // Do journal registers match the request's expectation? if b.req.CheckRegisters != nil && len(b.registers.Labels) != 0 && !b.req.CheckRegisters.Matches(b.registers) { b.resolved.status = pb.Status_REGISTER_MISMATCH b.state = stateError - } else if b.pln.spool.End != maxOffset && b.req.Offset == 0 && b.resolved.journalSpec.Flags.MayWrite() { + return + } + // Do we need to roll forward to the resumption offset of the journal? + if b.pln.spool.End < suspend.GetOffset() { + b.rollToOffset = suspend.GetOffset() + b.state = stateSendPipelineSync + return + } + // Does the synchronized offset not match the maximum of the fragment index? + if b.pln.spool.End != maxOffset && b.req.Offset == 0 && b.resolved.journalSpec.Flags.MayWrite() { b.resolved.status = pb.Status_INDEX_HAS_GREATER_OFFSET b.state = stateError - } else if b.req.Offset != 0 && b.req.Offset != maxOffset { + return + } + // Does the request have an explicit offset, which doesn't match the maximum of the index? + if b.req.Offset != 0 && b.req.Offset != maxOffset { // If a request offset is present, it must match |maxOffset|. b.resolved.status = pb.Status_WRONG_APPEND_OFFSET b.state = stateError - } else if b.req.Offset != 0 && b.pln.spool.End != maxOffset { - // Re-sync the pipeline at the explicitly requested |maxOffset|. + return + } + // Does the request have an explicit offset which matches the maximum of the index, + // but doesn't match the synchronized offset? + if b.req.Offset != 0 && b.pln.spool.End != maxOffset { + // Re-sync the pipeline at the explicitly requested `maxOffset`. b.rollToOffset = maxOffset b.state = stateSendPipelineSync - } else if b.pln.spool.Begin == indexMin { + return + } + + // At this point, we hold a synchronized and validated pipeline. + + // Do we need to update the suspension state of the journal? + switch b.req.Suspend { + case pb.AppendRequest_SUSPEND_RESUME: + if suspend.GetLevel() != pb.JournalSpec_Suspend_NONE { + b.nextSuspend = &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_NONE, + Offset: b.pln.spool.End, + } + b.state = stateUpdateSuspend + return + } + case pb.AppendRequest_SUSPEND_NO_RESUME: + if suspend.GetLevel() != pb.JournalSpec_Suspend_NONE { + b.resolved.status = pb.Status_SUSPENDED + b.state = stateError + return + } + case pb.AppendRequest_SUSPEND_IF_FLUSHED, pb.AppendRequest_SUSPEND_NOW: + // We're requested to suspend the journal. If it's not fully suspended + // but the index is completely empty, we can proceed to full suspension. + if suspend.GetLevel() != pb.JournalSpec_Suspend_FULL && indexMin == indexMax { + b.nextSuspend = &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_FULL, + Offset: b.pln.spool.End, + } + b.state = stateUpdateSuspend + return + } + // If the index is fully remote, or we're requested to suspend regardless, + // we can proceed to partially suspend the journal. + if suspend.GetLevel() == pb.JournalSpec_Suspend_NONE && + (!indexDirty || b.req.Suspend == pb.AppendRequest_SUSPEND_NOW) { + + b.nextSuspend = &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_PARTIAL, + Offset: b.pln.spool.End, + } + b.state = stateUpdateSuspend + + // The journal replication factor will be reduced, but this broker + // will likely still be the primary, which implies a dirty local spool + // may not be immediately rolled and persisted without deliberately + // rolling it forward here (this is relevant only for SUSPEND_NOW; + // SUSPEND_IF_FLUSHED's pre-condition is there are no local fragments). + b.rollToOffset = b.pln.spool.End + return + } + // If the journal was suspended, set SUSPENDED status to tell the caller as much. + // Note that fully suspended journals return SUSPENDED as part of onResolve(). + if suspend.GetLevel() == pb.JournalSpec_Suspend_PARTIAL { + b.resolved.status = pb.Status_SUSPENDED + } + } + + if b.pln.spool.Begin == indexMin { // The spool holds the journal's first known write and should be rolled. // This has the effect of "dirtying" the remote fragment index, // and protects against data loss if N > R consistency is lost (eg, Etcd fails). @@ -457,10 +546,32 @@ func (b *appendFSM) onValidatePreconditions() { // recovering brokers cannot distinguish this case from a newly-created // journal, which risks double-writes to journal offsets. b.rollToOffset = b.pln.spool.End - b.state = stateStreamContent + } + b.state = stateStreamContent +} + +func (b *appendFSM) onUpdateSuspend() { + b.mustState(stateUpdateSuspend) + + var update = *b.resolved.journalSpec + update.Suspend = b.nextSuspend + + addTrace(b.ctx, " ... updating suspension of JournalSpec to %v", update.Suspend) + b.readThroughRev, b.err = updateJournalSpec(b.ctx, b.resolved.item, b.resolved.assignments, update, b.svc.etcd) + addTrace(b.ctx, "updateJournalSpec() => %d, err: %v", b.readThroughRev, b.err) + + if b.err != nil { + b.err = errors.WithMessage(b.err, "updateJournalSpec") + b.state = stateError } else { - b.state = stateStreamContent + b.state = stateResolve } + + log.WithFields(log.Fields{ + "journal": update.Name, + "revision": b.readThroughRev, + "suspend": update.Suspend, + }).Info("updated journal suspension") } // onStreamContent is called with each received content message or error @@ -521,6 +632,10 @@ func (b *appendFSM) onStreamContent(req *pb.AppendRequest, err error) { } else if err == nil && !b.resolved.journalSpec.Flags.MayWrite() { // Non-empty appends cannot be made to non-writable journals. b.resolved.status = pb.Status_NOT_ALLOWED + } else if err == nil && + (b.req.Suspend == pb.AppendRequest_SUSPEND_IF_FLUSHED || b.req.Suspend == pb.AppendRequest_SUSPEND_NOW) { + // Appends that request a suspension may not have content. + b.resolved.status = pb.Status_NOT_ALLOWED } else if err == nil { // Regular content chunk. Forward it through the pipeline. b.pln.scatter(&pb.ReplicateRequest{ diff --git a/broker/append_fsm_test.go b/broker/append_fsm_test.go index 1dd56aff..8837def7 100644 --- a/broker/append_fsm_test.go +++ b/broker/append_fsm_test.go @@ -297,6 +297,80 @@ func TestFSMUpdateAssignments(t *testing.T) { peer.Cleanup() } +func TestFSMSuspendAndResume(t *testing.T) { + var ctx, etcd = context.Background(), etcdtest.TestClient() + defer etcdtest.Cleanup() + + var broker = newTestBroker(t, etcd, pb.ProcessSpec_ID{Zone: "local", Suffix: "broker"}) + setTestJournal(broker, pb.JournalSpec{Name: "a/journal", Replication: 1}, broker.id) + + // Case: Journal is eligible for partial suspension. + var fsm = newFSM(broker, ctx, pb.AppendRequest{ + Journal: "a/journal", + Offset: 1024, + Suspend: pb.AppendRequest_SUSPEND_NOW, + }) + fsm.onResolve() + fsm.resolved.replica.index.ReplaceRemote(fragment.CoverSet{ + {Fragment: pb.Fragment{Journal: "a/journal", Begin: 0, End: 1024}}, + }) + require.True(t, fsm.runTo(stateStreamContent)) + + require.Equal(t, pb.Status_SUSPENDED, fsm.resolved.status) + require.Equal(t, &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_PARTIAL, + Offset: 1024, + }, fsm.resolved.journalSpec.Suspend) + + require.NotNil(t, fsm.plnReturnCh) + fsm.returnPipeline() + + // Case: Journal is eligible for full suspension. + broker.initialFragmentLoad() // Clear remote fragments. + fsm = newFSM(broker, ctx, pb.AppendRequest{ + Journal: "a/journal", + Suspend: pb.AppendRequest_SUSPEND_IF_FLUSHED, + }) + require.False(t, fsm.runTo(stateStreamContent)) + + require.Equal(t, pb.Status_SUSPENDED, fsm.resolved.status) + require.Equal(t, &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_FULL, + Offset: 1024, + }, fsm.resolved.journalSpec.Suspend) + + require.NotNil(t, fsm.plnReturnCh) + fsm.returnPipeline() + + // Case: append doesn't wake the journal. + fsm = newFSM(broker, ctx, pb.AppendRequest{ + Journal: "a/journal", + Suspend: pb.AppendRequest_SUSPEND_NO_RESUME, + }) + require.False(t, fsm.runTo(stateStreamContent)) + require.Equal(t, pb.Status_SUSPENDED, fsm.resolved.status) + require.Nil(t, fsm.plnReturnCh) + + // Case: append DOES wake the journal. + fsm = newFSM(broker, ctx, pb.AppendRequest{ + Journal: "a/journal", + Suspend: pb.AppendRequest_SUSPEND_RESUME, + }) + require.True(t, fsm.runTo(stateStreamContent)) + + require.Equal(t, pb.Status_OK, fsm.resolved.status) + require.Equal(t, &pb.JournalSpec_Suspend{ + Level: pb.JournalSpec_Suspend_NONE, + Offset: 1024, + }, fsm.resolved.journalSpec.Suspend) + require.Equal(t, int64(1024), fsm.pln.spool.Begin) + + require.NotNil(t, fsm.plnReturnCh) + fsm.returnPipeline() + + broker.cleanup() +} + func TestFSMDesiredReplicas(t *testing.T) { var ctx, etcd = context.Background(), etcdtest.TestClient() defer etcdtest.Cleanup() diff --git a/broker/client/appender.go b/broker/client/appender.go index 264da2ac..86eccce2 100644 --- a/broker/client/appender.go +++ b/broker/client/appender.go @@ -8,6 +8,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" + log "github.com/sirupsen/logrus" pb "go.gazette.dev/core/broker/protocol" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -96,15 +97,21 @@ func (a *Appender) Close() (err error) { switch a.Response.Status { case pb.Status_OK: // Pass. + case pb.Status_INSUFFICIENT_JOURNAL_BROKERS: + err = ErrInsufficientJournalBrokers case pb.Status_JOURNAL_NOT_FOUND: err = ErrJournalNotFound + case pb.Status_NO_JOURNAL_PRIMARY_BROKER: + err = ErrNoJournalPrimaryBroker case pb.Status_NOT_JOURNAL_PRIMARY_BROKER: err = ErrNotJournalPrimaryBroker - case pb.Status_WRONG_APPEND_OFFSET: - err = ErrWrongAppendOffset case pb.Status_REGISTER_MISMATCH: err = errors.Wrapf(ErrRegisterMismatch, "selector %v doesn't match registers %v", a.Request.CheckRegisters, a.Response.Registers) + case pb.Status_SUSPENDED: + err = ErrSuspended + case pb.Status_WRONG_APPEND_OFFSET: + err = ErrWrongAppendOffset default: err = errors.New(a.Response.Status.String()) } @@ -182,16 +189,26 @@ func Append(ctx context.Context, rjc pb.RoutedJournalClient, req pb.AppendReques a.Abort() } + var squelch bool + if err == nil { return a.Response, nil } else if s, ok := status.FromError(err); ok && s.Code() == codes.Unavailable { // Fallthrough to retry - } else if err == ErrNotJournalPrimaryBroker || err == ErrInsufficientJournalBrokers { - // Fallthrough. + } else if err == ErrNotJournalPrimaryBroker || err == ErrNoJournalPrimaryBroker || err == ErrInsufficientJournalBrokers { + squelch = attempt == 0 // Fallthrough. } else { return a.Response, err } + if !squelch { + log.WithFields(log.Fields{ + "journal": req.Journal, + "err": err, + "attempt": attempt, + }).Warn("append failure (will retry)") + } + select { case <-ctx.Done(): return a.Response, ctx.Err() diff --git a/broker/client/reader.go b/broker/client/reader.go index 05418769..8e3ade79 100644 --- a/broker/client/reader.go +++ b/broker/client/reader.go @@ -195,6 +195,8 @@ func (r *Reader) Read(p []byte) (n int, err error) { err = ErrInsufficientJournalBrokers case pb.Status_OFFSET_NOT_YET_AVAILABLE: err = ErrOffsetNotYetAvailable + case pb.Status_SUSPENDED: + err = ErrSuspended default: err = errors.New(r.Response.Status.String()) } @@ -428,10 +430,12 @@ var ( // Map common broker error statuses into named errors. ErrInsufficientJournalBrokers = errors.New(pb.Status_INSUFFICIENT_JOURNAL_BROKERS.String()) ErrJournalNotFound = errors.New(pb.Status_JOURNAL_NOT_FOUND.String()) + ErrNoJournalPrimaryBroker = errors.New(pb.Status_NO_JOURNAL_PRIMARY_BROKER.String()) ErrNotJournalBroker = errors.New(pb.Status_NOT_JOURNAL_BROKER.String()) ErrNotJournalPrimaryBroker = errors.New(pb.Status_NOT_JOURNAL_PRIMARY_BROKER.String()) ErrOffsetNotYetAvailable = errors.New(pb.Status_OFFSET_NOT_YET_AVAILABLE.String()) ErrRegisterMismatch = errors.New(pb.Status_REGISTER_MISMATCH.String()) + ErrSuspended = errors.New(pb.Status_SUSPENDED.String()) ErrWrongAppendOffset = errors.New(pb.Status_WRONG_APPEND_OFFSET.String()) // ErrOffsetJump is returned by Reader.Read to indicate that the next byte diff --git a/broker/client/retry_reader.go b/broker/client/retry_reader.go index 00409720..e0b75679 100644 --- a/broker/client/retry_reader.go +++ b/broker/client/retry_reader.go @@ -112,6 +112,8 @@ func (rr *RetryReader) Read(p []byte) (n int, err error) { if rr.Reader.Request.Header != nil || attempt > 3 { return // Surface to caller. } + case ErrSuspended: + return // Surface to caller. case io.EOF: // EOF means we had an established RPC, but it might not have sent // any data before being closed server-side, so clear `attempts` to diff --git a/broker/fragment/index.go b/broker/fragment/index.go index 2ed7620b..9f812188 100644 --- a/broker/fragment/index.go +++ b/broker/fragment/index.go @@ -125,12 +125,13 @@ func (fi *Index) Query(ctx context.Context, req *pb.ReadRequest) (*pb.ReadRespon } } -// OffsetRange returns the [Begin, End) offset range of all Fragments in the index. -func (fi *Index) OffsetRange() (int64, int64) { +// OffsetRange returns the [Begin, End) offset range of all Fragments in the index, +// and a boolean indicating if the index has local-only fragments. +func (fi *Index) OffsetRange() (int64, int64, bool) { defer fi.mu.RUnlock() fi.mu.RLock() - return fi.set.BeginOffset(), fi.set.EndOffset() + return fi.set.BeginOffset(), fi.set.EndOffset(), len(fi.local) != 0 } // SpoolCommit adds local Spool Fragment |frag| to the index. diff --git a/broker/fragment/index_test.go b/broker/fragment/index_test.go index c32dc592..9a4d6c08 100644 --- a/broker/fragment/index_test.go +++ b/broker/fragment/index_test.go @@ -246,7 +246,7 @@ func (s *IndexSuite) TestBlockedContextCancelled(c *gc.C) { } func (s *IndexSuite) TestWalkStoresAndURLSigning(c *gc.C) { - var tmpdir, err = ioutil.TempDir("", "IndexSuite.TestWalkStores") + var tmpdir, err = os.MkdirTemp("", "IndexSuite.TestWalkStores") c.Assert(err, gc.IsNil) defer func() { os.RemoveAll(tmpdir) }() @@ -288,7 +288,7 @@ func (s *IndexSuite) TestWalkStoresAndURLSigning(c *gc.C) { <-ind.FirstRefreshCh() c.Check(ind.set, gc.HasLen, 3) - var bo, eo = ind.OffsetRange() + var bo, eo, _ = ind.OffsetRange() c.Check(bo, gc.Equals, int64(0x0)) c.Check(eo, gc.Equals, int64(0x255)) @@ -306,7 +306,7 @@ func (s *IndexSuite) TestWalkStoresAndURLSigning(c *gc.C) { ind.ReplaceRemote(set) c.Check(ind.set, gc.HasLen, 4) // Combined Fragments are reflected. - bo, eo = ind.OffsetRange() + bo, eo, _ = ind.OffsetRange() c.Check(bo, gc.Equals, int64(0x0)) c.Check(eo, gc.Equals, int64(0x555)) diff --git a/broker/protocol/journal_spec_extensions.go b/broker/protocol/journal_spec_extensions.go index 8e4cd147..28e6c737 100644 --- a/broker/protocol/journal_spec_extensions.go +++ b/broker/protocol/journal_spec_extensions.go @@ -2,7 +2,7 @@ package protocol import ( "fmt" - "io/ioutil" + "io" "math" "mime" "strings" @@ -113,6 +113,8 @@ func (m *JournalSpec) Validate() error { return ExtendContext(err, "Flags") } else if m.MaxAppendRate < 0 { return NewValidationError("invalid MaxAppendRate (%d; expected >= 0)", m.MaxAppendRate) + } else if err = m.Suspend.Validate(); err != nil { + return ExtendContext(err, "Suspend") } return nil } @@ -148,7 +150,7 @@ func (m *JournalSpec_Fragment) Validate() error { // error over a zero-valued struct having the proper shape. if tpl, err := template.New("postfix").Parse(m.PathPostfixTemplate); err != nil { return ExtendContext(NewValidationError(err.Error()), "PathPostfixTemplate") - } else if err = tpl.Execute(ioutil.Discard, struct { + } else if err = tpl.Execute(io.Discard, struct { Spool struct { Fragment FirstAppendTime time.Time @@ -164,6 +166,68 @@ func (m *JournalSpec_Fragment) Validate() error { return nil } +func (x JournalSpec_Suspend_Level) Validate() error { + switch x { + case JournalSpec_Suspend_NONE, JournalSpec_Suspend_PARTIAL, JournalSpec_Suspend_FULL: + return nil + default: + return NewValidationError("invalid Level variant (%s)", x) + } +} + +func (x JournalSpec_Suspend_Level) MarshalYAML() (interface{}, error) { + if s, ok := JournalSpec_Suspend_Level_name[int32(x)]; ok { + return s, nil + } else { + return int(x), nil + } +} + +func (x *JournalSpec_Suspend_Level) UnmarshalYAML(unmarshal func(interface{}) error) error { + // Directly map YAML integer to flag. + var i int + if err := unmarshal(&i); err == nil { + *x = JournalSpec_Suspend_Level(i) + return nil + } + // Otherwise, expect a YAML string which matches an enum name. + var str string + if err := unmarshal(&str); err != nil { + return err + } + if tag, ok := JournalSpec_Suspend_Level_value[str]; !ok { + return fmt.Errorf("%q is not a valid JournalSpec_Suspend_Level (options are %v)", str, JournalSpec_Suspend_Level_value) + } else { + *x = JournalSpec_Suspend_Level(tag) + return nil + } +} + +func (m *JournalSpec_Suspend) GetLevel() JournalSpec_Suspend_Level { + if m == nil { + return JournalSpec_Suspend_NONE + } else { + return m.Level + } +} + +func (m *JournalSpec_Suspend) GetOffset() int64 { + if m == nil { + return 0 + } else { + return m.Offset + } +} + +func (m *JournalSpec_Suspend) Validate() error { + if err := m.GetLevel().Validate(); err != nil { + return ExtendContext(err, "Level") + } else if m.GetOffset() < 0 { + return ExtendContext(NewValidationError("invalid Offset (%d; expected >= 0)", m.GetOffset()), "Offset") + } + return nil +} + // Validate returns an error if the JournalSpec_Flag is malformed. func (x JournalSpec_Flag) Validate() error { switch x { @@ -237,10 +301,18 @@ func (m *JournalSpec) MarshalString() string { // DesiredReplication returns the configured Replication of the spec. It // implements allocator.ItemValue. func (m *JournalSpec) DesiredReplication() int { - if MaxReplication < m.Replication { - return int(MaxReplication) + var r = m.Replication + + if m.Suspend.GetLevel() == JournalSpec_Suspend_PARTIAL { + r = 1 // Journal is suspended down to a single read-only replica. + } else if m.Suspend.GetLevel() == JournalSpec_Suspend_FULL { + r = 0 // Journal is suspended down to zero replicas. } - return int(m.Replication) + + if r > MaxReplication { + r = MaxReplication + } + return int(r) } // UnionJournalSpecs returns a JournalSpec combining all non-zero-valued fields @@ -317,6 +389,8 @@ func IntersectJournalSpecs(a, b JournalSpec) JournalSpec { if a.MaxAppendRate != b.MaxAppendRate { a.MaxAppendRate = 0 } + a.Suspend = nil + return a } diff --git a/broker/protocol/journal_spec_extensions_test.go b/broker/protocol/journal_spec_extensions_test.go index 27a99835..f3e3b6e5 100644 --- a/broker/protocol/journal_spec_extensions_test.go +++ b/broker/protocol/journal_spec_extensions_test.go @@ -82,9 +82,17 @@ func (s *JournalSuite) TestSpecValidationCases(c *gc.C) { spec.Replication = 1024 c.Check(spec.Validate(), gc.ErrorMatches, `invalid Replication \(1024; .*`) spec.Replication = 3 + c.Check(spec.DesiredReplication(), gc.Equals, 3) - // DesiredReplication honors the lower-bound of the spec & global limit. + // Suspension level modulates down the desired replication. + spec.Suspend = &JournalSpec_Suspend{Level: JournalSpec_Suspend_PARTIAL} + c.Check(spec.DesiredReplication(), gc.Equals, 1) + spec.Suspend.Level = JournalSpec_Suspend_FULL + c.Check(spec.DesiredReplication(), gc.Equals, 0) + spec.Suspend.Level = JournalSpec_Suspend_NONE c.Check(spec.DesiredReplication(), gc.Equals, 3) + + // DesiredReplication honors the lower-bound of the spec & global limit. defer func(r int32) { MaxReplication = r }(MaxReplication) MaxReplication = 1 c.Check(spec.DesiredReplication(), gc.Equals, 1) @@ -175,6 +183,15 @@ func (s *JournalSuite) TestSpecValidationCases(c *gc.C) { f.Stores = append(f.Stores, "invalid") c.Check(f.Validate(), gc.ErrorMatches, `Stores\[2\]: not absolute \(invalid\)`) + f.Stores = f.Stores[:1] + + spec.Suspend.Offset = -1 + c.Check(spec.Validate(), gc.ErrorMatches, `Suspend.Offset: invalid Offset \(-1; expected >= 0\)`) + spec.Suspend.Offset = 1234 + spec.Suspend.Level = 12345 + c.Check(spec.Validate(), gc.ErrorMatches, `Suspend.Level: invalid Level variant \(12345\)`) + spec.Suspend.Level = JournalSpec_Suspend_FULL + c.Check(spec.Validate(), gc.IsNil) } func (s *JournalSuite) TestMetaLabelExtraction(c *gc.C) { diff --git a/broker/protocol/protocol.pb.go b/broker/protocol/protocol.pb.go index b69dcbe1..be52066e 100644 --- a/broker/protocol/protocol.pb.go +++ b/broker/protocol/protocol.pb.go @@ -79,6 +79,11 @@ const ( // The Append is refused because a registers selector was provided with the // request, but it was not matched by current register values of the journal. Status_REGISTER_MISMATCH Status = 13 + // The operation cannot complete because the journal has been suspended. + // This implicitly means the journal has no available content to read. + // An Append will resume the journal, or the client may want to filter + // further requests directed to suspended journals. + Status_SUSPENDED Status = 14 ) var Status_name = map[int32]string{ @@ -96,6 +101,7 @@ var Status_name = map[int32]string{ 11: "WRONG_APPEND_OFFSET", 12: "INDEX_HAS_GREATER_OFFSET", 13: "REGISTER_MISMATCH", + 14: "SUSPENDED", } var Status_value = map[string]int32{ @@ -113,6 +119,7 @@ var Status_value = map[string]int32{ "WRONG_APPEND_OFFSET": 11, "INDEX_HAS_GREATER_OFFSET": 12, "REGISTER_MISMATCH": 13, + "SUSPENDED": 14, } func (x Status) String() string { @@ -220,6 +227,91 @@ func (JournalSpec_Flag) EnumDescriptor() ([]byte, []int) { return fileDescriptor_0c0999e5af553218, []int{3, 0} } +type JournalSpec_Suspend_Level int32 + +const ( + // When NONE, the journal is not suspended and is fully replicated. + JournalSpec_Suspend_NONE JournalSpec_Suspend_Level = 0 + // When PARTIAL, the journal is scaled down to a single replica + // which monitors the fragment index and serves reads. + JournalSpec_Suspend_PARTIAL JournalSpec_Suspend_Level = 1 + // When FULL, the journal's fragment index MUST be empty and the journal + // is scaled down to zero replicas. Operations other than Append, + // which resumes the journal, will fail with status SUSPENDED. + // + // Clients should identify and filter requests which would otherwise be + // directed at suspended journals. For example, a client may watch a + // journal listing and filter to read journals which have a suspension + // level of ACTIVE or PARTIAL. + JournalSpec_Suspend_FULL JournalSpec_Suspend_Level = 2 +) + +var JournalSpec_Suspend_Level_name = map[int32]string{ + 0: "NONE", + 1: "PARTIAL", + 2: "FULL", +} + +var JournalSpec_Suspend_Level_value = map[string]int32{ + "NONE": 0, + "PARTIAL": 1, + "FULL": 2, +} + +func (x JournalSpec_Suspend_Level) String() string { + return proto.EnumName(JournalSpec_Suspend_Level_name, int32(x)) +} + +func (JournalSpec_Suspend_Level) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_0c0999e5af553218, []int{3, 1, 0} +} + +// Suspension control for this request. +type AppendRequest_Suspend int32 + +const ( + // SUSPEND_RESUME (the default) will resume a suspended journal. + AppendRequest_SUSPEND_RESUME AppendRequest_Suspend = 0 + // SUSPEND_NO_RESUME fails with status SUSPENDED if the journal is suspended. + AppendRequest_SUSPEND_NO_RESUME AppendRequest_Suspend = 1 + // SUSPEND_IF_FLUSHED potentially suspends the requested journal: + // - If the journal has local content which is not yet available in the + // remote fragment store, the operation has no effect. + // - If the journal has not content at all, it is fully suspended. + // - Otherwise, the journal is partially suspended. + AppendRequest_SUSPEND_IF_FLUSHED AppendRequest_Suspend = 2 + // SUSPEND_NOW is similar to SUSPEND_IF_FLUSHED, but will also + // partially suspend the journal even if it has local content which + // is not yet available in the remote fragment store. + // + // This operation is always safe -- even once suspended, all former replicas + // persist content to the remote store as per usual -- but it can result in + // many small files if a journal is repeatedly suspended and resumed. + AppendRequest_SUSPEND_NOW AppendRequest_Suspend = 3 +) + +var AppendRequest_Suspend_name = map[int32]string{ + 0: "SUSPEND_RESUME", + 1: "SUSPEND_NO_RESUME", + 2: "SUSPEND_IF_FLUSHED", + 3: "SUSPEND_NOW", +} + +var AppendRequest_Suspend_value = map[string]int32{ + "SUSPEND_RESUME": 0, + "SUSPEND_NO_RESUME": 1, + "SUSPEND_IF_FLUSHED": 2, + "SUSPEND_NOW": 3, +} + +func (x AppendRequest_Suspend) String() string { + return proto.EnumName(AppendRequest_Suspend_name, int32(x)) +} + +func (AppendRequest_Suspend) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_0c0999e5af553218, []int{10, 0} +} + // Label defines a key & value pair which can be attached to entities like // JournalSpecs and BrokerSpecs. Labels may be used to provide identifying // attributes which do not directly imply semantics to the core system, but @@ -369,7 +461,8 @@ type JournalSpec struct { // be processed. If zero (the default), no rate limiting is applied. A global // rate limit still may be in effect, in which case the effective rate is the // smaller of the journal vs global rate. - MaxAppendRate int64 `protobuf:"varint,7,opt,name=max_append_rate,json=maxAppendRate,proto3" json:"max_append_rate,omitempty" yaml:"max_append_rate,omitempty"` + MaxAppendRate int64 `protobuf:"varint,7,opt,name=max_append_rate,json=maxAppendRate,proto3" json:"max_append_rate,omitempty" yaml:"max_append_rate,omitempty"` + Suspend *JournalSpec_Suspend `protobuf:"bytes,8,opt,name=suspend,proto3" json:"suspend,omitempty" yaml:",omitempty"` } func (m *JournalSpec) Reset() { *m = JournalSpec{} } @@ -510,6 +603,53 @@ func (m *JournalSpec_Fragment) XXX_DiscardUnknown() { var xxx_messageInfo_JournalSpec_Fragment proto.InternalMessageInfo +// Suspension control for this journal. +// If unset the suspension level is implicitly NONE, with an offset of zero. +// +// The Suspend field is managed by Gazette, and is updated when an Append RPC +// suspends a journal or if auto-suspension is enabled. Operators should +// not set it directly. However when utilizing suspension, operators MUST +// take care to pass-through Suspend when applying updates to JournalSpecs. +type JournalSpec_Suspend struct { + Level JournalSpec_Suspend_Level `protobuf:"varint,1,opt,name=level,proto3,enum=protocol.JournalSpec_Suspend_Level" json:"level,omitempty" yaml:",omitempty"` + // The lower-bound journal offset at which appends should proceed, + // once this journal is resumed. + Offset int64 `protobuf:"varint,2,opt,name=offset,proto3" json:"offset,omitempty" yaml:",omitempty"` +} + +func (m *JournalSpec_Suspend) Reset() { *m = JournalSpec_Suspend{} } +func (m *JournalSpec_Suspend) String() string { return proto.CompactTextString(m) } +func (*JournalSpec_Suspend) ProtoMessage() {} +func (*JournalSpec_Suspend) Descriptor() ([]byte, []int) { + return fileDescriptor_0c0999e5af553218, []int{3, 1} +} +func (m *JournalSpec_Suspend) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *JournalSpec_Suspend) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + if deterministic { + return xxx_messageInfo_JournalSpec_Suspend.Marshal(b, m, deterministic) + } else { + b = b[:cap(b)] + n, err := m.MarshalToSizedBuffer(b) + if err != nil { + return nil, err + } + return b[:n], nil + } +} +func (m *JournalSpec_Suspend) XXX_Merge(src proto.Message) { + xxx_messageInfo_JournalSpec_Suspend.Merge(m, src) +} +func (m *JournalSpec_Suspend) XXX_Size() int { + return m.ProtoSize() +} +func (m *JournalSpec_Suspend) XXX_DiscardUnknown() { + xxx_messageInfo_JournalSpec_Suspend.DiscardUnknown(m) +} + +var xxx_messageInfo_JournalSpec_Suspend proto.InternalMessageInfo + // ProcessSpec describes a uniquely identified process and its addressable // endpoint. type ProcessSpec struct { @@ -950,7 +1090,8 @@ type AppendRequest struct { UnionRegisters *LabelSet `protobuf:"bytes,7,opt,name=union_registers,json=unionRegisters,proto3" json:"union_registers,omitempty"` // Labels to subtract from current registers if the RPC succeeds and appended // at least one byte. - SubtractRegisters *LabelSet `protobuf:"bytes,8,opt,name=subtract_registers,json=subtractRegisters,proto3" json:"subtract_registers,omitempty"` + SubtractRegisters *LabelSet `protobuf:"bytes,8,opt,name=subtract_registers,json=subtractRegisters,proto3" json:"subtract_registers,omitempty"` + Suspend AppendRequest_Suspend `protobuf:"varint,9,opt,name=suspend,proto3,enum=protocol.AppendRequest_Suspend" json:"suspend,omitempty"` // Content chunks to be appended. Immediately prior to closing the stream, // the client must send an empty chunk (eg, zero-valued AppendRequest) to // indicate the Append should be committed. Absence of this empty chunk @@ -1713,6 +1854,10 @@ func init() { golang_proto.RegisterEnum("protocol.CompressionCodec", CompressionCodec_name, CompressionCodec_value) proto.RegisterEnum("protocol.JournalSpec_Flag", JournalSpec_Flag_name, JournalSpec_Flag_value) golang_proto.RegisterEnum("protocol.JournalSpec_Flag", JournalSpec_Flag_name, JournalSpec_Flag_value) + proto.RegisterEnum("protocol.JournalSpec_Suspend_Level", JournalSpec_Suspend_Level_name, JournalSpec_Suspend_Level_value) + golang_proto.RegisterEnum("protocol.JournalSpec_Suspend_Level", JournalSpec_Suspend_Level_name, JournalSpec_Suspend_Level_value) + proto.RegisterEnum("protocol.AppendRequest_Suspend", AppendRequest_Suspend_name, AppendRequest_Suspend_value) + golang_proto.RegisterEnum("protocol.AppendRequest_Suspend", AppendRequest_Suspend_name, AppendRequest_Suspend_value) proto.RegisterType((*Label)(nil), "protocol.Label") golang_proto.RegisterType((*Label)(nil), "protocol.Label") proto.RegisterType((*LabelSet)(nil), "protocol.LabelSet") @@ -1723,6 +1868,8 @@ func init() { golang_proto.RegisterType((*JournalSpec)(nil), "protocol.JournalSpec") proto.RegisterType((*JournalSpec_Fragment)(nil), "protocol.JournalSpec.Fragment") golang_proto.RegisterType((*JournalSpec_Fragment)(nil), "protocol.JournalSpec.Fragment") + proto.RegisterType((*JournalSpec_Suspend)(nil), "protocol.JournalSpec.Suspend") + golang_proto.RegisterType((*JournalSpec_Suspend)(nil), "protocol.JournalSpec.Suspend") proto.RegisterType((*ProcessSpec)(nil), "protocol.ProcessSpec") golang_proto.RegisterType((*ProcessSpec)(nil), "protocol.ProcessSpec") proto.RegisterType((*ProcessSpec_ID)(nil), "protocol.ProcessSpec.ID") @@ -1777,173 +1924,183 @@ func init() { } var fileDescriptor_0c0999e5af553218 = []byte{ - // 2646 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4d, 0x70, 0xdb, 0xc6, - 0x15, 0x16, 0x48, 0x90, 0x04, 0x1f, 0x49, 0x09, 0xda, 0xc4, 0xb6, 0xcc, 0xc4, 0xa2, 0xc2, 0xfc, - 0x54, 0x76, 0x12, 0xca, 0x91, 0x3b, 0x49, 0xea, 0x4e, 0xda, 0x90, 0x22, 0x65, 0xd3, 0xa1, 0x49, - 0xce, 0x92, 0x8a, 0xe3, 0x1c, 0x8a, 0x81, 0x80, 0x15, 0x85, 0x0a, 0x04, 0x50, 0x00, 0x74, 0xa4, - 0xdc, 0x72, 0x6a, 0x0e, 0xe9, 0x4c, 0xa7, 0xa7, 0x9c, 0xda, 0x5c, 0x7a, 0x6e, 0x67, 0x7a, 0x6b, - 0x2f, 0x3d, 0xba, 0xb7, 0x1c, 0x3b, 0xd3, 0x56, 0x9d, 0xc6, 0x33, 0x9d, 0x9e, 0x7d, 0xcc, 0xa9, - 0xb3, 0x3f, 0x20, 0x21, 0xfe, 0x58, 0xf6, 0xc1, 0x17, 0xcd, 0xee, 0x7b, 0xdf, 0xbe, 0xdd, 0x7d, - 0xef, 0xf1, 0x7b, 0x6f, 0x21, 0x58, 0xdf, 0xf7, 0xdd, 0x23, 0xe2, 0x6f, 0x79, 0xbe, 0x1b, 0xba, - 0x86, 0x6b, 0x8f, 0x07, 0x15, 0x36, 0x40, 0x4a, 0x34, 0x2f, 0xbe, 0x38, 0x70, 0x07, 0x2e, 0x9b, - 0x6d, 0xd1, 0x11, 0xd7, 0x17, 0xd7, 0x07, 0xae, 0x3b, 0xb0, 0x09, 0x5f, 0xb6, 0x3f, 0x3a, 0xd8, - 0x32, 0x47, 0xbe, 0x1e, 0x5a, 0xae, 0xc3, 0xf5, 0xe5, 0x03, 0x48, 0xb5, 0xf4, 0x7d, 0x62, 0x23, - 0x04, 0xb2, 0xa3, 0x0f, 0xc9, 0x9a, 0xb4, 0x21, 0x6d, 0x66, 0x31, 0x1b, 0xa3, 0x17, 0x21, 0xf5, - 0x40, 0xb7, 0x47, 0x64, 0x2d, 0xc1, 0x84, 0x7c, 0x82, 0xde, 0x86, 0xb4, 0xe7, 0x93, 0x03, 0xeb, - 0x78, 0x2d, 0xb9, 0x21, 0x6d, 0x2a, 0xb5, 0x0b, 0x8f, 0x4f, 0x4b, 0xab, 0x27, 0xfa, 0xd0, 0xbe, - 0x59, 0x7e, 0xcb, 0x1d, 0x5a, 0x21, 0x19, 0x7a, 0xe1, 0x49, 0x19, 0x0b, 0xd0, 0x4d, 0xf9, 0x7f, - 0xdf, 0x94, 0xa4, 0x72, 0x1f, 0x14, 0xb6, 0x4f, 0x8f, 0x84, 0xa8, 0x06, 0x69, 0x9b, 0x8e, 0x83, - 0x35, 0x69, 0x23, 0xb9, 0x99, 0xdb, 0x5e, 0xa9, 0x8c, 0x2f, 0xc5, 0x30, 0xb5, 0xcb, 0x0f, 0x4f, - 0x4b, 0x4b, 0x0b, 0xac, 0xf2, 0x95, 0xc2, 0xea, 0x17, 0x12, 0x14, 0x84, 0x59, 0x9b, 0x18, 0xa1, - 0xeb, 0xa3, 0x6d, 0xc8, 0x58, 0x8e, 0x61, 0x8f, 0x4c, 0x7e, 0x93, 0xdc, 0x36, 0x9a, 0x32, 0xde, - 0x23, 0x61, 0x4d, 0xa6, 0xf6, 0x71, 0x04, 0xa4, 0x6b, 0xc8, 0x31, 0x5f, 0x93, 0x38, 0x6f, 0x8d, - 0x00, 0xde, 0x54, 0xbe, 0xfe, 0xa6, 0xb4, 0xc4, 0xce, 0xf0, 0x55, 0x16, 0x72, 0x77, 0xdc, 0x91, - 0xef, 0xe8, 0x76, 0xcf, 0x23, 0x06, 0xfa, 0x61, 0xdc, 0x91, 0xb5, 0x8d, 0xb9, 0xd7, 0xf8, 0xfe, - 0xb4, 0x94, 0x11, 0x6b, 0x84, 0xab, 0xdf, 0x83, 0x9c, 0x4f, 0x3c, 0xdb, 0x32, 0x58, 0x70, 0xd8, - 0x39, 0x52, 0x8b, 0x3c, 0x1b, 0x47, 0xa2, 0xee, 0xd8, 0x99, 0xc9, 0x85, 0x67, 0x7f, 0x8d, 0x9e, - 0xfd, 0xdb, 0xd3, 0x92, 0xf4, 0xf8, 0xb4, 0xb4, 0x36, 0x6d, 0xef, 0x2d, 0xcb, 0xb1, 0x2d, 0x87, - 0x8c, 0x5d, 0x8b, 0xf6, 0x40, 0x39, 0xf0, 0xf5, 0xc1, 0x90, 0x38, 0xe1, 0x9a, 0xcc, 0x6c, 0xae, - 0x4f, 0x6c, 0xc6, 0x6e, 0x5a, 0xd9, 0x15, 0xa8, 0x27, 0xc5, 0x6b, 0x6c, 0x0a, 0xfd, 0x14, 0x52, - 0x07, 0xb6, 0x3e, 0x08, 0xd6, 0xd2, 0x1b, 0xd2, 0x66, 0xa1, 0x76, 0x75, 0x91, 0x63, 0xd4, 0xd8, - 0x16, 0xda, 0xae, 0xad, 0x0f, 0x30, 0x5f, 0x87, 0x5a, 0xb0, 0x32, 0xd4, 0x8f, 0x35, 0xdd, 0xf3, - 0x88, 0x63, 0x6a, 0xbe, 0x1e, 0x92, 0xb5, 0xcc, 0x86, 0xb4, 0x99, 0xac, 0xbd, 0xf6, 0xf8, 0xb4, - 0xb4, 0xc1, 0x4d, 0x4d, 0x01, 0xe2, 0x27, 0x29, 0x0c, 0xf5, 0xe3, 0x2a, 0x53, 0x61, 0x3d, 0x24, - 0xc5, 0xaf, 0x52, 0xa0, 0x44, 0x17, 0xa0, 0x29, 0x6d, 0x13, 0x67, 0x10, 0x1e, 0xb2, 0xa8, 0x25, - 0x17, 0xa6, 0x34, 0x07, 0x21, 0x17, 0x56, 0x0d, 0x77, 0xe8, 0xf9, 0x24, 0x08, 0x2c, 0xd7, 0xd1, - 0x0c, 0xd7, 0x24, 0x06, 0x0b, 0xd9, 0xf2, 0x76, 0x71, 0xe2, 0xaa, 0x9d, 0x09, 0x64, 0x87, 0x22, - 0x6a, 0x6f, 0x3c, 0x3e, 0x2d, 0x95, 0xb9, 0xd5, 0x99, 0xe5, 0xf1, 0x6d, 0x54, 0x63, 0x6a, 0x25, - 0xfa, 0x09, 0xa4, 0x83, 0xd0, 0xf5, 0x09, 0x0d, 0x72, 0x72, 0x33, 0xcb, 0x2c, 0xcd, 0x75, 0x5e, - 0x21, 0xba, 0x52, 0x8f, 0xc2, 0xb1, 0x58, 0x85, 0x02, 0x50, 0x7d, 0x72, 0xe0, 0x93, 0xe0, 0x50, - 0xb3, 0x9c, 0x90, 0xf8, 0x0f, 0x74, 0x5b, 0x84, 0xf6, 0x72, 0x85, 0x13, 0x44, 0x25, 0x22, 0x88, - 0x4a, 0x5d, 0x10, 0x44, 0xed, 0x6d, 0x11, 0xd5, 0x57, 0xf8, 0x46, 0xd3, 0x06, 0x62, 0x1b, 0x7f, - 0xfd, 0xef, 0x92, 0x84, 0x57, 0x04, 0xa0, 0x29, 0xf4, 0xe8, 0x63, 0xc8, 0xfa, 0x24, 0x24, 0x0e, - 0x4b, 0xe8, 0xd4, 0x79, 0xbb, 0x5d, 0x59, 0x98, 0x43, 0xcc, 0xfa, 0xc4, 0x14, 0x1a, 0xc2, 0xf2, - 0x81, 0x3d, 0x8a, 0x5f, 0x25, 0x7d, 0x9e, 0xf1, 0x37, 0x85, 0xf1, 0x12, 0x37, 0x7e, 0x76, 0xf9, - 0xf4, 0x56, 0x05, 0xa6, 0x1e, 0x5f, 0xe3, 0x67, 0x70, 0xc1, 0xd3, 0xc3, 0x43, 0xcd, 0x73, 0x83, - 0xf0, 0xc0, 0x3a, 0xd6, 0x28, 0xd4, 0x8e, 0x92, 0x2f, 0x5b, 0xbb, 0xf6, 0xf8, 0xb4, 0xf4, 0x06, - 0x37, 0x3b, 0x17, 0x16, 0x0f, 0xec, 0x0b, 0x14, 0xd1, 0xe5, 0x80, 0xbe, 0xd0, 0x0b, 0x26, 0xab, - 0x82, 0x4c, 0x73, 0x1d, 0xad, 0x42, 0xa1, 0xdd, 0xe9, 0x6b, 0xbd, 0x6e, 0x63, 0xa7, 0xb9, 0xdb, - 0x6c, 0xd4, 0xd5, 0x25, 0x94, 0x07, 0xa5, 0xa3, 0xe1, 0x7a, 0xa7, 0xdd, 0xba, 0xaf, 0x4a, 0x7c, - 0x76, 0x0f, 0xb3, 0x59, 0x02, 0x01, 0xa4, 0xa9, 0xee, 0x1e, 0x56, 0x65, 0x61, 0xe8, 0xf7, 0x12, - 0xe4, 0xba, 0xbe, 0x6b, 0x90, 0x20, 0x60, 0x74, 0x54, 0x81, 0x84, 0x65, 0x0a, 0x2e, 0x5c, 0x9b, - 0x24, 0x67, 0x0c, 0x52, 0x69, 0xd6, 0x05, 0xbb, 0x25, 0x2c, 0x13, 0x6d, 0x82, 0x42, 0x1c, 0xd3, - 0x73, 0x2d, 0x27, 0xe4, 0xb4, 0x5f, 0xcb, 0x7f, 0x7f, 0x5a, 0x52, 0x1a, 0x42, 0x86, 0xc7, 0xda, - 0xe2, 0xbb, 0x90, 0x68, 0xd6, 0x69, 0xdd, 0xf8, 0xdc, 0x75, 0xc6, 0x75, 0x83, 0x8e, 0xd1, 0x45, - 0x48, 0x07, 0xa3, 0x03, 0x5a, 0x21, 0x78, 0xe1, 0x10, 0x33, 0x7e, 0xc2, 0x9b, 0xf2, 0x97, 0xf4, - 0x9c, 0xbf, 0x94, 0x00, 0x6a, 0xac, 0xb6, 0xb1, 0x63, 0xf6, 0x21, 0xef, 0xf1, 0x23, 0x69, 0x81, - 0x47, 0x0c, 0x71, 0xe0, 0x0b, 0x73, 0x0f, 0x5c, 0x2b, 0xc6, 0xf8, 0x6c, 0x59, 0xe4, 0x4b, 0xc4, - 0x62, 0x39, 0x2f, 0x76, 0xf9, 0x57, 0xa1, 0xf0, 0x73, 0xce, 0x26, 0x9a, 0x6d, 0x0d, 0x2d, 0x7e, - 0xa3, 0x02, 0xce, 0x0b, 0x61, 0x8b, 0xca, 0xca, 0xff, 0x48, 0xc4, 0x98, 0xe0, 0x75, 0xc8, 0x08, - 0xa5, 0x20, 0xf0, 0x5c, 0x9c, 0xab, 0x23, 0x1d, 0xda, 0x80, 0xd4, 0x3e, 0x19, 0x58, 0x9c, 0xa8, - 0x93, 0x35, 0xf8, 0xfe, 0xb4, 0x94, 0xee, 0x1c, 0x1c, 0x04, 0x24, 0xc4, 0x5c, 0x81, 0x5e, 0x86, - 0x24, 0x71, 0x4c, 0x46, 0xca, 0x67, 0xf5, 0x54, 0x8c, 0xae, 0x42, 0x32, 0x18, 0x0d, 0xc5, 0x6f, - 0x70, 0x75, 0x72, 0xcb, 0xde, 0xed, 0xea, 0x3b, 0xbd, 0xd1, 0x50, 0xc4, 0x83, 0x62, 0xd0, 0xad, - 0x79, 0x64, 0x93, 0x3a, 0x8f, 0x6c, 0xe6, 0x90, 0xc8, 0xbb, 0x50, 0xd8, 0xd7, 0x8d, 0x23, 0xcb, - 0x19, 0x68, 0x8c, 0x16, 0xd8, 0xcf, 0x26, 0x5b, 0x5b, 0x9d, 0xa5, 0x8d, 0xbc, 0xc0, 0xb1, 0x19, - 0xba, 0x0c, 0xca, 0xd0, 0x35, 0xb5, 0xd0, 0x1a, 0x0a, 0xc2, 0xc5, 0x99, 0xa1, 0x6b, 0xf6, 0xad, - 0x21, 0x41, 0xaf, 0x40, 0x3e, 0x9e, 0xf4, 0x6b, 0x0a, 0x0b, 0x77, 0x2e, 0x96, 0xe6, 0xe5, 0x8f, - 0x20, 0x23, 0x2e, 0x45, 0xdb, 0x09, 0x4f, 0xf7, 0xc3, 0x77, 0x98, 0x67, 0xd3, 0x98, 0x4f, 0x22, - 0xe9, 0x36, 0x73, 0xa5, 0x90, 0x6e, 0x47, 0xd2, 0x1b, 0xcc, 0x81, 0x19, 0x2e, 0xbd, 0x51, 0xfe, - 0x63, 0x02, 0x72, 0x98, 0xe8, 0x26, 0x26, 0xbf, 0x18, 0x91, 0x20, 0x44, 0x9b, 0x90, 0x3e, 0x24, - 0xba, 0x49, 0x7c, 0x91, 0x2f, 0xea, 0xc4, 0x21, 0xb7, 0x99, 0x1c, 0x0b, 0x7d, 0x3c, 0xae, 0x89, - 0x27, 0xc4, 0xb5, 0x0c, 0x69, 0x97, 0x85, 0x69, 0x4e, 0xe0, 0x84, 0x86, 0x1e, 0x6d, 0xdf, 0x76, - 0x8d, 0x23, 0x16, 0x3d, 0x05, 0xf3, 0x09, 0xda, 0x80, 0xbc, 0xe9, 0x6a, 0x8e, 0x1b, 0x6a, 0x9e, - 0xef, 0x1e, 0x9f, 0xb0, 0x08, 0x29, 0x18, 0x4c, 0xb7, 0xed, 0x86, 0x5d, 0x2a, 0xa1, 0xc9, 0x38, - 0x24, 0xa1, 0x6e, 0xea, 0xa1, 0xae, 0xb9, 0x8e, 0x7d, 0xc2, 0xfc, 0xaf, 0xe0, 0x7c, 0x24, 0xec, - 0x38, 0xf6, 0x09, 0xba, 0x0a, 0x40, 0x8b, 0x97, 0x38, 0x44, 0x66, 0xe6, 0x10, 0x59, 0xe2, 0x98, - 0x7c, 0x88, 0x5e, 0x83, 0x65, 0x96, 0x6a, 0xda, 0x38, 0x3a, 0x0a, 0x8b, 0x4e, 0x9e, 0x49, 0xef, - 0xf2, 0x10, 0x95, 0x7f, 0x9b, 0x80, 0x3c, 0x77, 0x59, 0xe0, 0xb9, 0x4e, 0x40, 0xa8, 0xcf, 0x82, - 0x50, 0x0f, 0x47, 0x01, 0xf3, 0xd9, 0x72, 0xdc, 0x67, 0x3d, 0x26, 0xc7, 0x42, 0x1f, 0xf3, 0x6e, - 0xe2, 0x1c, 0xef, 0x3e, 0x8d, 0xdb, 0xae, 0x02, 0x7c, 0xe6, 0x5b, 0x21, 0xd1, 0xe8, 0x1a, 0xe6, - 0xbb, 0xa9, 0x9b, 0x31, 0x2d, 0x35, 0x8c, 0x2a, 0xb1, 0x0e, 0x24, 0x35, 0xdd, 0xd5, 0x44, 0xa9, - 0x1a, 0x6b, 0x2d, 0x5e, 0x81, 0x7c, 0x34, 0xd6, 0x46, 0x3e, 0xaf, 0x07, 0x59, 0x9c, 0x8b, 0x64, - 0x7b, 0xbe, 0x8d, 0xd6, 0x20, 0x63, 0xb8, 0x0e, 0x2d, 0x21, 0xcc, 0xa9, 0x79, 0x1c, 0x4d, 0xcb, - 0x5f, 0x26, 0xa1, 0x20, 0xfa, 0x82, 0xe7, 0x95, 0x55, 0xd3, 0xb9, 0x91, 0x9c, 0xc9, 0x8d, 0x89, - 0x03, 0x53, 0x0b, 0x1d, 0xf8, 0x21, 0xac, 0x18, 0x87, 0xc4, 0x38, 0xd2, 0x7c, 0x32, 0xb0, 0x82, - 0x90, 0xf8, 0x81, 0x28, 0x7c, 0x97, 0x66, 0x5a, 0x3e, 0xde, 0x0c, 0xe3, 0x65, 0x86, 0xc7, 0x11, - 0x1c, 0xfd, 0x18, 0x56, 0x46, 0x0e, 0x25, 0x91, 0x89, 0x85, 0xcc, 0xa2, 0xa6, 0x11, 0x2f, 0x33, - 0xe8, 0x64, 0x71, 0x15, 0x50, 0x30, 0xda, 0x0f, 0x7d, 0xdd, 0x08, 0x63, 0xeb, 0x95, 0x85, 0xeb, - 0x57, 0x23, 0xf4, 0xc4, 0x44, 0x2c, 0x08, 0xf2, 0x99, 0x20, 0x88, 0xda, 0xf5, 0x9b, 0x04, 0x2c, - 0x47, 0xa1, 0x78, 0xe6, 0x6c, 0xad, 0x9c, 0x97, 0xad, 0x82, 0x54, 0xa3, 0xd8, 0x5d, 0x83, 0xb4, - 0xe1, 0x0e, 0x69, 0x51, 0x48, 0x2e, 0x4c, 0x31, 0x81, 0x40, 0xd7, 0x69, 0x2b, 0x13, 0x5d, 0x59, - 0x5e, 0x78, 0xe5, 0x09, 0x88, 0xa6, 0x64, 0xe8, 0x86, 0xba, 0xad, 0x19, 0x87, 0x23, 0xe7, 0x28, - 0xe0, 0x61, 0xc5, 0x39, 0x26, 0xdb, 0x61, 0x22, 0xf4, 0x3a, 0x2c, 0x9b, 0xc4, 0xd6, 0x4f, 0x88, - 0x19, 0x81, 0xd2, 0x0c, 0x54, 0x10, 0x52, 0x0e, 0x2b, 0xff, 0x25, 0x01, 0x2a, 0x16, 0x0d, 0x3f, - 0x79, 0xf6, 0x14, 0xad, 0x00, 0x7d, 0x22, 0x7a, 0x6e, 0xa0, 0xdb, 0x4f, 0xb8, 0xe8, 0x18, 0x73, - 0xf6, 0xaa, 0x99, 0xa7, 0xb9, 0xea, 0x06, 0xe4, 0x74, 0xe3, 0xc8, 0x71, 0x3f, 0xb3, 0x89, 0x39, - 0x20, 0x82, 0xd5, 0xe2, 0x22, 0x74, 0x13, 0x90, 0x49, 0x3c, 0x9f, 0xd0, 0x1b, 0x98, 0xda, 0x13, - 0x7e, 0x31, 0xab, 0x13, 0x98, 0x10, 0x2d, 0xce, 0x19, 0xca, 0xa7, 0x62, 0xa8, 0x99, 0xc4, 0x0e, - 0x75, 0xe1, 0xe3, 0xbc, 0x10, 0xd6, 0xa9, 0xac, 0xfc, 0x37, 0x09, 0x56, 0x63, 0xde, 0x7b, 0x8e, - 0x1c, 0x18, 0x27, 0xad, 0xe4, 0x53, 0x90, 0xd6, 0x33, 0xe7, 0x54, 0xf9, 0x77, 0x12, 0xe4, 0x5a, - 0x56, 0x10, 0x46, 0x49, 0xf0, 0x23, 0x50, 0x02, 0xf1, 0x53, 0x17, 0x69, 0xb0, 0x88, 0x09, 0x44, - 0xea, 0x8f, 0xe1, 0xb4, 0x86, 0x7d, 0xa6, 0x87, 0xc6, 0x61, 0x54, 0xc3, 0xd8, 0x04, 0xdd, 0x80, - 0x3c, 0x1b, 0x68, 0x3e, 0x09, 0x46, 0x43, 0x22, 0xb8, 0x77, 0xf6, 0xca, 0x39, 0x86, 0xc2, 0x0c, - 0x74, 0x47, 0x56, 0x12, 0x6a, 0xf2, 0x8e, 0xac, 0x24, 0x55, 0xb9, 0xfc, 0xdf, 0x04, 0xe4, 0xf9, - 0x09, 0x9f, 0xfb, 0xcf, 0xf7, 0x43, 0x50, 0x44, 0x22, 0xf1, 0x47, 0xd1, 0x99, 0x57, 0x6a, 0xfc, - 0x0c, 0xd1, 0x93, 0x35, 0xf2, 0x41, 0xb4, 0xaa, 0xf8, 0x27, 0x09, 0xa2, 0xc4, 0x43, 0x5b, 0x20, - 0xcf, 0x6f, 0x3b, 0x63, 0x8f, 0x51, 0x61, 0x80, 0x01, 0xe9, 0xef, 0x9b, 0x96, 0x5d, 0x9f, 0x3c, - 0xb0, 0x82, 0xe8, 0xc1, 0x9e, 0xc4, 0xb9, 0xa1, 0x6b, 0x62, 0x21, 0x42, 0x6f, 0x42, 0xca, 0x77, - 0x47, 0x21, 0x11, 0xd9, 0x10, 0xfb, 0xca, 0x81, 0xa9, 0x58, 0x98, 0xe3, 0x18, 0xf4, 0x03, 0x58, - 0x31, 0x7c, 0xa2, 0x87, 0x64, 0x62, 0x92, 0x95, 0x48, 0xbc, 0xcc, 0xc5, 0x91, 0xd5, 0x3b, 0xb2, - 0x22, 0xab, 0xa9, 0xf2, 0x3f, 0x25, 0xc8, 0x57, 0x3d, 0xcf, 0x3e, 0x89, 0x72, 0xe1, 0x03, 0xc8, - 0x18, 0x87, 0xba, 0x33, 0x20, 0xd1, 0x47, 0x95, 0x2b, 0x93, 0xed, 0xe2, 0xc0, 0xca, 0x0e, 0x43, - 0x45, 0x9f, 0x33, 0xc4, 0x9a, 0xe2, 0x57, 0x12, 0xa4, 0xb9, 0x06, 0x55, 0xe0, 0x05, 0x72, 0xec, - 0x11, 0x23, 0xd4, 0xce, 0x5c, 0x90, 0x3d, 0x8c, 0xf1, 0x2a, 0x57, 0xdd, 0x8d, 0x5d, 0xf3, 0x6d, - 0x48, 0x8f, 0xbc, 0x80, 0xf8, 0xa1, 0x08, 0xdc, 0x7c, 0xe7, 0x61, 0x01, 0x42, 0xaf, 0x42, 0xda, - 0x24, 0x36, 0x11, 0x6e, 0x99, 0xfa, 0xfd, 0x0b, 0x55, 0xd9, 0x62, 0x25, 0x99, 0x1e, 0xfa, 0x79, - 0xe7, 0x51, 0xf9, 0x5f, 0x09, 0x50, 0xa3, 0x5f, 0x67, 0xf0, 0xdc, 0x3a, 0x80, 0xd9, 0x5e, 0x2d, - 0x39, 0xdb, 0xab, 0xd1, 0x3e, 0x81, 0x36, 0x7f, 0x63, 0x0c, 0xcf, 0x00, 0xda, 0x10, 0x46, 0x88, - 0x37, 0x60, 0xc5, 0x21, 0xc7, 0xa1, 0xe6, 0xe9, 0x03, 0xa2, 0x85, 0xee, 0x11, 0x71, 0x04, 0xeb, - 0x15, 0xa8, 0xb8, 0xab, 0x0f, 0x48, 0x9f, 0x0a, 0xd1, 0x15, 0x00, 0x06, 0xe1, 0xaf, 0x1e, 0x4a, - 0xc9, 0x29, 0x9c, 0xa5, 0x12, 0xf6, 0xe4, 0x41, 0xb7, 0x20, 0x1f, 0x58, 0x03, 0x47, 0x0f, 0x47, - 0x3e, 0xe9, 0xf7, 0x5b, 0x82, 0xe7, 0x9f, 0xf0, 0x80, 0x56, 0x1e, 0x9e, 0x96, 0x24, 0xf6, 0x3a, - 0x3e, 0xb3, 0x70, 0xa6, 0xb3, 0x51, 0xa6, 0x3b, 0x9b, 0xf2, 0x9f, 0x13, 0xb0, 0x1a, 0xf3, 0xef, - 0x73, 0xe7, 0x85, 0x26, 0x64, 0x23, 0x8a, 0x8d, 0x88, 0xe1, 0xf5, 0x59, 0x1e, 0x1e, 0x9f, 0xa4, - 0xa2, 0x8d, 0xbf, 0x62, 0x71, 0x3b, 0x93, 0xd5, 0xf3, 0x9c, 0x2d, 0xcf, 0x71, 0x76, 0xf1, 0x13, - 0xc8, 0x8e, 0xad, 0xa0, 0xb7, 0xce, 0x30, 0xc9, 0x9c, 0x12, 0x70, 0x86, 0x46, 0xae, 0x00, 0x50, - 0x7f, 0x12, 0x93, 0xf5, 0xad, 0xfc, 0xb5, 0x9c, 0xe5, 0x92, 0x3d, 0xdf, 0x2e, 0xff, 0x4a, 0x82, - 0x14, 0x23, 0x0b, 0xf4, 0x3e, 0x64, 0x86, 0x64, 0xb8, 0x4f, 0x6b, 0x05, 0xff, 0x7d, 0x9f, 0xf7, - 0x96, 0x8f, 0xe0, 0xb4, 0x80, 0x7a, 0xbe, 0x35, 0xd4, 0xfd, 0x13, 0xfe, 0x55, 0x11, 0x47, 0x53, - 0x74, 0x0d, 0xb2, 0xd1, 0x63, 0x3e, 0xfa, 0xb0, 0x74, 0xf6, 0xad, 0x3f, 0x51, 0x8b, 0x06, 0xed, - 0x0f, 0x09, 0x48, 0x73, 0xaf, 0xa3, 0x0f, 0x00, 0xa2, 0x07, 0xfb, 0x53, 0x7f, 0x5f, 0xc8, 0x8a, - 0x15, 0x4d, 0x73, 0x42, 0x8e, 0x89, 0xa7, 0x20, 0xc7, 0x2d, 0x90, 0x49, 0x68, 0x98, 0x82, 0x48, - 0x2f, 0x4c, 0x67, 0x40, 0xa5, 0x11, 0x1a, 0x66, 0xe4, 0x56, 0x0a, 0x2c, 0x7e, 0x21, 0x81, 0x4c, - 0x85, 0xd4, 0xbf, 0x86, 0x3d, 0xa2, 0xe5, 0x33, 0x3a, 0xa5, 0x8c, 0xb3, 0x42, 0xd2, 0x34, 0xd1, - 0x4b, 0x90, 0xe5, 0x6e, 0xa2, 0xda, 0x04, 0xd3, 0x2a, 0x5c, 0xd0, 0x34, 0x51, 0x11, 0x94, 0x31, - 0xfb, 0xf1, 0x5f, 0xeb, 0x78, 0x4e, 0x17, 0xfa, 0xfa, 0x41, 0xa8, 0x85, 0xc4, 0xe7, 0xaf, 0x78, - 0x19, 0x2b, 0x54, 0xd0, 0x27, 0xfe, 0x30, 0xfa, 0xcc, 0x41, 0xff, 0x5e, 0xfb, 0x2e, 0x01, 0x69, - 0x9e, 0xd1, 0x28, 0x0d, 0x89, 0xce, 0x47, 0xea, 0x12, 0xba, 0x00, 0xab, 0x77, 0x3a, 0x7b, 0xb8, - 0x5d, 0x6d, 0x69, 0xed, 0x4e, 0x5f, 0xdb, 0xed, 0xec, 0xb5, 0xeb, 0xaa, 0x84, 0xae, 0xc0, 0xe5, - 0x76, 0x47, 0x8b, 0x34, 0x5d, 0xdc, 0xbc, 0x5b, 0xc5, 0xf7, 0xb5, 0x1a, 0xee, 0x7c, 0xd4, 0xc0, - 0x6a, 0x02, 0xad, 0x43, 0x91, 0xa2, 0x17, 0xe8, 0x93, 0xe8, 0x22, 0xa0, 0xb8, 0x5e, 0xc8, 0x53, - 0x68, 0x03, 0x5e, 0x6e, 0xb6, 0x7b, 0x7b, 0xbb, 0xbb, 0xcd, 0x9d, 0x66, 0xa3, 0x3d, 0x0d, 0xe8, - 0xa9, 0x32, 0x7a, 0x19, 0xd6, 0x3a, 0xbb, 0xbb, 0xbd, 0x46, 0x9f, 0x1d, 0xe7, 0x7e, 0xa3, 0xaf, - 0x55, 0x3f, 0xae, 0x36, 0x5b, 0xd5, 0x5a, 0xab, 0xa1, 0xa6, 0xd1, 0x0a, 0xe4, 0xee, 0xe1, 0x4e, - 0xfb, 0x96, 0x86, 0x3b, 0x7b, 0xfd, 0x86, 0x9a, 0xa1, 0xc7, 0xef, 0xe2, 0x4e, 0xb7, 0xd3, 0xab, - 0xb6, 0xb4, 0xbb, 0xcd, 0xde, 0xdd, 0x6a, 0x7f, 0xe7, 0xb6, 0xaa, 0xa0, 0x97, 0xe0, 0x52, 0xa3, - 0xbf, 0x53, 0xd7, 0xfa, 0xb8, 0xda, 0xee, 0x55, 0x77, 0xfa, 0xcd, 0x4e, 0x5b, 0xdb, 0xad, 0x36, - 0x5b, 0x8d, 0xba, 0x9a, 0xa5, 0x46, 0xa8, 0xed, 0x6a, 0xab, 0xd5, 0xb9, 0xd7, 0xa8, 0xab, 0x80, - 0x2e, 0xc1, 0x0b, 0xdc, 0x6a, 0xb5, 0xdb, 0x6d, 0xb4, 0xeb, 0x1a, 0x3f, 0x80, 0x9a, 0xa3, 0x87, - 0x69, 0xb6, 0xeb, 0x8d, 0x4f, 0xb4, 0xdb, 0xd5, 0x9e, 0x76, 0x0b, 0x37, 0xaa, 0xfd, 0x06, 0x8e, - 0xb4, 0x79, 0xba, 0x37, 0x6e, 0xdc, 0x6a, 0xf6, 0xa8, 0x70, 0xbc, 0x77, 0xe1, 0x9a, 0x03, 0xea, - 0xf4, 0xf7, 0x0f, 0x94, 0x83, 0x4c, 0xb3, 0xfd, 0x71, 0xb5, 0xd5, 0xac, 0xab, 0x4b, 0x48, 0x01, - 0xb9, 0xdd, 0x69, 0x37, 0x54, 0x89, 0x8e, 0x6e, 0x7d, 0xda, 0xec, 0xaa, 0x09, 0x54, 0x80, 0xec, - 0xa7, 0xbd, 0x7e, 0xb5, 0x5d, 0xaf, 0xe2, 0xba, 0x9a, 0x44, 0x00, 0xe9, 0x5e, 0xbb, 0xda, 0xed, - 0xde, 0x57, 0x65, 0xea, 0x6b, 0x0a, 0xa2, 0xfb, 0xb6, 0x3a, 0xd5, 0xba, 0x56, 0x6f, 0xec, 0x74, - 0xee, 0x76, 0x71, 0xa3, 0xd7, 0x6b, 0x76, 0xda, 0x6a, 0x6a, 0xfb, 0xcb, 0xe4, 0xa4, 0x73, 0x78, - 0x0f, 0x64, 0xda, 0x6d, 0xa0, 0x0b, 0xd3, 0xdd, 0x07, 0xab, 0x24, 0xc5, 0x8b, 0xf3, 0x9b, 0x92, - 0xeb, 0x12, 0x7a, 0x1f, 0x52, 0xac, 0xc6, 0xa1, 0x8b, 0xf3, 0x2b, 0x75, 0xf1, 0xd2, 0x8c, 0x5c, - 0x90, 0xe7, 0x7b, 0x20, 0xd3, 0x17, 0x7d, 0x7c, 0xcb, 0xd8, 0x47, 0x91, 0xf8, 0x96, 0xf1, 0x87, - 0xff, 0x75, 0x09, 0x7d, 0x00, 0x69, 0xfe, 0xbc, 0x42, 0x67, 0x6d, 0x4f, 0xde, 0xbe, 0xc5, 0xb5, - 0x59, 0x05, 0x5f, 0xbe, 0x29, 0xa1, 0xdb, 0x90, 0x1d, 0xb7, 0xd2, 0xa8, 0x18, 0xdf, 0xe5, 0xec, - 0xeb, 0xa4, 0xf8, 0xd2, 0x5c, 0x5d, 0x64, 0xe7, 0x3a, 0xb5, 0x54, 0xa0, 0xde, 0x18, 0xb3, 0x71, - 0xdc, 0xda, 0x74, 0x31, 0x8e, 0x5b, 0x9b, 0xa1, 0xef, 0x5a, 0xe3, 0xe1, 0x7f, 0xd6, 0x97, 0x1e, - 0x7e, 0xb7, 0x2e, 0x7d, 0xfb, 0xdd, 0xba, 0xf4, 0xeb, 0x47, 0xeb, 0x4b, 0xdf, 0x3c, 0x5a, 0x97, - 0xfe, 0xfa, 0x68, 0x5d, 0xfa, 0xf6, 0xd1, 0xfa, 0xd2, 0xdf, 0x1f, 0xad, 0x2f, 0x7d, 0xfa, 0xea, - 0xc0, 0xad, 0x0c, 0xf4, 0xcf, 0x49, 0x18, 0x92, 0x8a, 0x49, 0x1e, 0x6c, 0x19, 0xae, 0x4f, 0xb6, - 0xa6, 0xfe, 0xab, 0xb6, 0x9f, 0x66, 0xa3, 0x1b, 0xff, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x7b, 0xe9, - 0xe8, 0x1d, 0x6f, 0x1b, 0x00, 0x00, + // 2809 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x59, 0x4d, 0x6c, 0x1b, 0xc7, + 0xf5, 0xd7, 0xf2, 0x73, 0xf9, 0x48, 0x4a, 0xab, 0x49, 0x6c, 0xd3, 0x4c, 0x2c, 0x2a, 0x74, 0x92, + 0xbf, 0xec, 0x24, 0x94, 0x23, 0xff, 0x91, 0x0f, 0xff, 0x91, 0x7f, 0x43, 0x8a, 0x94, 0x4d, 0x87, + 0x22, 0x89, 0x21, 0x15, 0xc7, 0x01, 0xda, 0xc5, 0x6a, 0x77, 0x44, 0xb1, 0x5a, 0xee, 0xb2, 0xbb, + 0x4b, 0x47, 0xca, 0x2d, 0xa7, 0xe6, 0x90, 0x02, 0x45, 0x4f, 0xe9, 0xa5, 0x0d, 0x0a, 0xf4, 0xdc, + 0x02, 0xbd, 0xa5, 0x97, 0x1e, 0xdd, 0x5b, 0x8e, 0x05, 0xda, 0xaa, 0x68, 0x0c, 0x14, 0xbd, 0xd6, + 0xc7, 0x9c, 0x8a, 0xf9, 0x22, 0x57, 0xfc, 0xb0, 0x9c, 0x83, 0x2f, 0xc4, 0xce, 0x7b, 0xbf, 0x79, + 0xf3, 0xe6, 0xcd, 0x9b, 0xf7, 0xde, 0x3c, 0xc2, 0xda, 0xbe, 0xe7, 0x1e, 0x11, 0x6f, 0x73, 0xe8, + 0xb9, 0x81, 0x6b, 0xba, 0xf6, 0xf8, 0xa3, 0xc4, 0x3e, 0x90, 0x2a, 0xc7, 0xf9, 0xe7, 0x7b, 0x6e, + 0xcf, 0x65, 0xa3, 0x4d, 0xfa, 0xc5, 0xf9, 0xf9, 0xb5, 0x9e, 0xeb, 0xf6, 0x6c, 0xc2, 0xa7, 0xed, + 0x8f, 0x0e, 0x36, 0xad, 0x91, 0x67, 0x04, 0x7d, 0xd7, 0xe1, 0xfc, 0xe2, 0x01, 0xc4, 0x1b, 0xc6, + 0x3e, 0xb1, 0x11, 0x82, 0x98, 0x63, 0x0c, 0x48, 0x4e, 0x59, 0x57, 0x36, 0x52, 0x98, 0x7d, 0xa3, + 0xe7, 0x21, 0xfe, 0xc0, 0xb0, 0x47, 0x24, 0x17, 0x61, 0x44, 0x3e, 0x40, 0x6f, 0x40, 0x62, 0xe8, + 0x91, 0x83, 0xfe, 0x71, 0x2e, 0xba, 0xae, 0x6c, 0xa8, 0x95, 0x0b, 0x8f, 0x4f, 0x0b, 0xab, 0x27, + 0xc6, 0xc0, 0xbe, 0x55, 0x7c, 0xdd, 0x1d, 0xf4, 0x03, 0x32, 0x18, 0x06, 0x27, 0x45, 0x2c, 0x40, + 0xb7, 0x62, 0xff, 0xfe, 0xaa, 0xa0, 0x14, 0xbb, 0xa0, 0xb2, 0x75, 0x3a, 0x24, 0x40, 0x15, 0x48, + 0xd8, 0xf4, 0xdb, 0xcf, 0x29, 0xeb, 0xd1, 0x8d, 0xf4, 0xd6, 0x4a, 0x69, 0xbc, 0x29, 0x86, 0xa9, + 0x5c, 0x7e, 0x78, 0x5a, 0x58, 0x5a, 0x20, 0x95, 0xcf, 0x14, 0x52, 0x3f, 0x53, 0x20, 0x2b, 0xc4, + 0xda, 0xc4, 0x0c, 0x5c, 0x0f, 0x6d, 0x41, 0xb2, 0xef, 0x98, 0xf6, 0xc8, 0xe2, 0x3b, 0x49, 0x6f, + 0xa1, 0x29, 0xe1, 0x1d, 0x12, 0x54, 0x62, 0x54, 0x3e, 0x96, 0x40, 0x3a, 0x87, 0x1c, 0xf3, 0x39, + 0x91, 0xf3, 0xe6, 0x08, 0xe0, 0x2d, 0xf5, 0xcb, 0xaf, 0x0a, 0x4b, 0x4c, 0x87, 0x5f, 0xa6, 0x21, + 0x7d, 0xd7, 0x1d, 0x79, 0x8e, 0x61, 0x77, 0x86, 0xc4, 0x44, 0xff, 0x1b, 0x36, 0x64, 0x65, 0x7d, + 0xee, 0x36, 0xbe, 0x3b, 0x2d, 0x24, 0xc5, 0x1c, 0x61, 0xea, 0xb7, 0x21, 0xed, 0x91, 0xa1, 0xdd, + 0x37, 0xd9, 0xe1, 0x30, 0x3d, 0xe2, 0x8b, 0x2c, 0x1b, 0x46, 0xa2, 0xf6, 0xd8, 0x98, 0xd1, 0x85, + 0xba, 0xbf, 0x4c, 0x75, 0xff, 0xe6, 0xb4, 0xa0, 0x3c, 0x3e, 0x2d, 0xe4, 0xa6, 0xe5, 0xbd, 0xde, + 0x77, 0xec, 0xbe, 0x43, 0xc6, 0xa6, 0x45, 0x7b, 0xa0, 0x1e, 0x78, 0x46, 0x6f, 0x40, 0x9c, 0x20, + 0x17, 0x63, 0x32, 0xd7, 0x26, 0x32, 0x43, 0x3b, 0x2d, 0xed, 0x08, 0xd4, 0x93, 0xce, 0x6b, 0x2c, + 0x0a, 0xfd, 0x00, 0xe2, 0x07, 0xb6, 0xd1, 0xf3, 0x73, 0x89, 0x75, 0x65, 0x23, 0x5b, 0xb9, 0xb6, + 0xc8, 0x30, 0x5a, 0x68, 0x09, 0x7d, 0xc7, 0x36, 0x7a, 0x98, 0xcf, 0x43, 0x0d, 0x58, 0x19, 0x18, + 0xc7, 0xba, 0x31, 0x1c, 0x12, 0xc7, 0xd2, 0x3d, 0x23, 0x20, 0xb9, 0xe4, 0xba, 0xb2, 0x11, 0xad, + 0xbc, 0xfc, 0xf8, 0xb4, 0xb0, 0xce, 0x45, 0x4d, 0x01, 0xc2, 0x9a, 0x64, 0x07, 0xc6, 0x71, 0x99, + 0xb1, 0xb0, 0x11, 0x10, 0xd4, 0x84, 0xa4, 0x3f, 0xf2, 0xe9, 0x30, 0xa7, 0xb2, 0x4d, 0x5e, 0x99, + 0xbf, 0xc9, 0x0e, 0x07, 0x2d, 0x3a, 0x0b, 0x29, 0x24, 0xff, 0x45, 0x1c, 0x54, 0x69, 0x10, 0x7a, + 0x45, 0x6c, 0xe2, 0xf4, 0x82, 0x43, 0xe6, 0x05, 0xd1, 0x85, 0x57, 0x84, 0x83, 0x90, 0x0b, 0xab, + 0xa6, 0x3b, 0x18, 0x7a, 0xc4, 0xf7, 0xfb, 0xae, 0xa3, 0x9b, 0xae, 0x45, 0x4c, 0xe6, 0x02, 0xcb, + 0x5b, 0xf9, 0x89, 0x56, 0xdb, 0x13, 0xc8, 0x36, 0x45, 0x54, 0x5e, 0x7d, 0x7c, 0x5a, 0x28, 0x72, + 0xa9, 0x33, 0xd3, 0xc3, 0xcb, 0x68, 0xe6, 0xd4, 0x4c, 0xf4, 0xff, 0x90, 0xf0, 0x03, 0xd7, 0x23, + 0xd4, 0x69, 0xa2, 0x1b, 0x29, 0x26, 0x69, 0xee, 0x61, 0x64, 0xe5, 0x96, 0x3a, 0x14, 0x8e, 0xc5, + 0x2c, 0xe4, 0x83, 0xe6, 0x91, 0x03, 0x8f, 0xf8, 0x87, 0x7a, 0xdf, 0x09, 0x88, 0xf7, 0xc0, 0xb0, + 0x85, 0xab, 0x5c, 0x2e, 0xf1, 0x80, 0x53, 0x92, 0x01, 0xa7, 0x54, 0x15, 0x01, 0xa7, 0xf2, 0x86, + 0xf0, 0x92, 0x97, 0xf8, 0x42, 0xd3, 0x02, 0x42, 0x0b, 0x7f, 0xf9, 0x8f, 0x82, 0x82, 0x57, 0x04, + 0xa0, 0x2e, 0xf8, 0xe8, 0x43, 0x48, 0x79, 0x24, 0x20, 0x0e, 0xbb, 0x20, 0xf1, 0xf3, 0x56, 0xbb, + 0xb2, 0xd0, 0x27, 0x99, 0xf4, 0x89, 0x28, 0x34, 0x80, 0xe5, 0x03, 0x7b, 0x14, 0xde, 0x4a, 0xe2, + 0x3c, 0xe1, 0xaf, 0x09, 0xe1, 0x05, 0x2e, 0xfc, 0xec, 0xf4, 0xe9, 0xa5, 0xb2, 0x8c, 0x3d, 0xde, + 0xc6, 0x8f, 0xe0, 0xc2, 0xd0, 0x08, 0x0e, 0xf5, 0xa1, 0xeb, 0x07, 0x07, 0xfd, 0x63, 0x9d, 0x42, + 0x6d, 0xe9, 0xcc, 0xa9, 0xca, 0xf5, 0xc7, 0xa7, 0x85, 0x57, 0xb9, 0xd8, 0xb9, 0xb0, 0xf0, 0xc1, + 0x3e, 0x47, 0x11, 0x6d, 0x0e, 0xe8, 0x0a, 0x3e, 0x8f, 0x8c, 0xf9, 0xaf, 0x15, 0x48, 0x0a, 0xd7, + 0x45, 0x6d, 0x88, 0xdb, 0xe4, 0x01, 0xb1, 0x99, 0x33, 0x2e, 0x6f, 0x5d, 0x7d, 0xa2, 0xa3, 0x97, + 0x1a, 0x14, 0xba, 0xc8, 0x63, 0xb9, 0x20, 0xea, 0xdf, 0xee, 0xc1, 0x81, 0x4f, 0x02, 0xe6, 0xa5, + 0x8b, 0xfd, 0x9b, 0x83, 0x8a, 0x1b, 0x10, 0x67, 0x52, 0x91, 0x0a, 0xb1, 0x66, 0xab, 0x59, 0xd3, + 0x96, 0x50, 0x1a, 0x92, 0xed, 0x32, 0xee, 0xd6, 0xcb, 0x0d, 0x4d, 0xa1, 0xe4, 0x9d, 0xbd, 0x46, + 0x43, 0x8b, 0x88, 0xb0, 0x5e, 0x86, 0x18, 0xbd, 0xf8, 0x68, 0x15, 0xb2, 0xcd, 0x56, 0x57, 0xef, + 0xb4, 0x6b, 0xdb, 0xf5, 0x9d, 0x7a, 0xad, 0xaa, 0x2d, 0xa1, 0x0c, 0xa8, 0x2d, 0x1d, 0x57, 0x5b, + 0xcd, 0xc6, 0x7d, 0x4d, 0xe1, 0xa3, 0x7b, 0x98, 0x8d, 0x22, 0x08, 0x20, 0x41, 0x79, 0xf7, 0xb0, + 0x16, 0x13, 0x82, 0x7e, 0xab, 0x40, 0xba, 0xed, 0xb9, 0x26, 0xf1, 0x7d, 0x16, 0x9b, 0x4b, 0x10, + 0xe9, 0x5b, 0x22, 0x31, 0xe4, 0x26, 0x66, 0x08, 0x41, 0x4a, 0xf5, 0xaa, 0x08, 0xf5, 0x91, 0xbe, + 0x85, 0x36, 0x40, 0x25, 0x8e, 0x35, 0x74, 0xfb, 0x0e, 0xdf, 0x69, 0xaa, 0x92, 0xf9, 0xee, 0xb4, + 0xa0, 0xd6, 0x04, 0x0d, 0x8f, 0xb9, 0xf9, 0xb7, 0x20, 0x52, 0xaf, 0xd2, 0x24, 0xfa, 0xa9, 0xeb, + 0x8c, 0x93, 0x28, 0xfd, 0x46, 0x17, 0x21, 0xe1, 0x8f, 0x0e, 0x68, 0xba, 0xe4, 0x59, 0x54, 0x8c, + 0xb8, 0x86, 0xb7, 0x62, 0x9f, 0x53, 0x3d, 0x7f, 0xaa, 0x00, 0x54, 0x58, 0xa2, 0x67, 0x6a, 0x76, + 0x21, 0x33, 0xe4, 0x2a, 0xe9, 0xfe, 0x90, 0x98, 0x42, 0xe1, 0x0b, 0x73, 0x15, 0xae, 0xe4, 0x43, + 0xc1, 0x7d, 0x59, 0x9c, 0x81, 0x0c, 0xe9, 0xe9, 0x61, 0x68, 0xf3, 0x57, 0x21, 0xfb, 0x63, 0x7e, + 0xde, 0xba, 0xdd, 0x1f, 0xf4, 0xf9, 0x8e, 0xb2, 0x38, 0x23, 0x88, 0x0d, 0x4a, 0x2b, 0xfe, 0x35, + 0x12, 0x0a, 0x63, 0xaf, 0x40, 0x52, 0x30, 0x45, 0x36, 0x4b, 0x87, 0x13, 0x97, 0xe4, 0xa1, 0x75, + 0x88, 0xef, 0x93, 0x5e, 0xdf, 0x11, 0xce, 0x00, 0xdf, 0x9d, 0x16, 0x12, 0x2d, 0x76, 0xf2, 0x98, + 0x33, 0xd0, 0x8b, 0x10, 0xa5, 0x81, 0x36, 0x3a, 0xc3, 0xa7, 0x64, 0x74, 0x0d, 0xa2, 0xfe, 0x68, + 0x20, 0x02, 0xc8, 0xea, 0x64, 0x97, 0x9d, 0x3b, 0xe5, 0x37, 0x3b, 0xa3, 0x81, 0x38, 0x0f, 0x8a, + 0x41, 0xb7, 0xe7, 0x45, 0xca, 0xf8, 0x79, 0x91, 0x72, 0x4e, 0x04, 0x7c, 0x0b, 0xb2, 0xfb, 0x86, + 0x79, 0xd4, 0x77, 0x7a, 0x3a, 0x8b, 0x69, 0xec, 0xce, 0xa7, 0x2a, 0xab, 0xb3, 0x31, 0x2f, 0x23, + 0x70, 0x6c, 0x84, 0x2e, 0x83, 0x3a, 0x70, 0x2d, 0x3d, 0xe8, 0x0f, 0x44, 0xf6, 0xc1, 0xc9, 0x81, + 0x6b, 0x75, 0xfb, 0x03, 0x82, 0x5e, 0x82, 0x4c, 0xf8, 0xc6, 0xb2, 0xb4, 0x92, 0xc2, 0xe9, 0xd0, + 0x1d, 0x2d, 0x7e, 0x00, 0x49, 0xb1, 0x29, 0x5a, 0x5b, 0x0d, 0x0d, 0x2f, 0x78, 0x93, 0x59, 0x36, + 0x81, 0xf9, 0x40, 0x52, 0xb7, 0x98, 0x29, 0x05, 0x75, 0x4b, 0x52, 0x6f, 0x32, 0x03, 0x26, 0x39, + 0xf5, 0x66, 0xf1, 0xf7, 0x11, 0x48, 0x63, 0x62, 0x58, 0x98, 0xfc, 0x64, 0x44, 0xfc, 0x00, 0x6d, + 0x40, 0xe2, 0x90, 0x18, 0x16, 0xf1, 0x84, 0xbf, 0x68, 0x13, 0x83, 0xdc, 0x61, 0x74, 0x2c, 0xf8, + 0xe1, 0x73, 0x8d, 0x3c, 0xe1, 0x5c, 0x8b, 0xe3, 0x5b, 0x3e, 0x7b, 0x70, 0x82, 0x43, 0x55, 0xdb, + 0xb7, 0x5d, 0xf3, 0x88, 0x9d, 0x9e, 0x8a, 0xf9, 0x00, 0xad, 0x43, 0xc6, 0x72, 0x75, 0xc7, 0x0d, + 0xf4, 0xa1, 0xe7, 0x1e, 0x9f, 0xb0, 0x13, 0x52, 0x31, 0x58, 0x6e, 0xd3, 0x0d, 0xda, 0x94, 0x42, + 0x9d, 0x71, 0x40, 0x02, 0xc3, 0x32, 0x02, 0x43, 0x77, 0x1d, 0xfb, 0x84, 0xd9, 0x5f, 0xc5, 0x19, + 0x49, 0x6c, 0x39, 0xf6, 0x09, 0xba, 0x06, 0x40, 0x33, 0xb9, 0x50, 0x22, 0x39, 0xa3, 0x44, 0x8a, + 0x38, 0x16, 0xff, 0x44, 0x2f, 0xc3, 0x32, 0x73, 0x35, 0x7d, 0x7c, 0x3a, 0x2a, 0x3b, 0x9d, 0x0c, + 0xa3, 0xee, 0xf2, 0x23, 0x2a, 0xfe, 0x2a, 0x02, 0x19, 0x6e, 0x32, 0x7f, 0xe8, 0x3a, 0x3e, 0xa1, + 0x36, 0xf3, 0x03, 0x23, 0x18, 0xf9, 0x22, 0x36, 0x86, 0x6c, 0xd6, 0x61, 0x74, 0x2c, 0xf8, 0x21, + 0xeb, 0x46, 0xce, 0xb1, 0xee, 0xd3, 0x98, 0xed, 0x1a, 0xc0, 0x27, 0x5e, 0x3f, 0x20, 0x3a, 0x9d, + 0xc3, 0x6c, 0x37, 0xb5, 0x33, 0xc6, 0xa5, 0x82, 0x51, 0x29, 0x54, 0x8e, 0xc5, 0xa7, 0x4b, 0x3c, + 0xe9, 0xaa, 0xa1, 0x3a, 0xeb, 0x25, 0xc8, 0xc8, 0x6f, 0x7d, 0xe4, 0xf1, 0x64, 0x96, 0xc2, 0x69, + 0x49, 0xdb, 0xf3, 0x6c, 0x94, 0x83, 0xa4, 0xe9, 0x3a, 0x34, 0xff, 0x31, 0xa3, 0x66, 0xb0, 0x1c, + 0x16, 0x7f, 0x13, 0x83, 0xac, 0x28, 0x92, 0x9e, 0x95, 0x57, 0x4d, 0xfb, 0x46, 0x74, 0xc6, 0x37, + 0x26, 0x06, 0x8c, 0x2f, 0x34, 0xe0, 0xfb, 0xb0, 0x62, 0x1e, 0x12, 0xf3, 0x48, 0xf7, 0x48, 0xaf, + 0xef, 0x07, 0xc4, 0xf3, 0x45, 0xd6, 0xbe, 0x34, 0x53, 0xff, 0xf2, 0x97, 0x01, 0x5e, 0x66, 0x78, + 0x2c, 0xe1, 0xe8, 0xff, 0x60, 0x65, 0xe4, 0xd0, 0x20, 0x32, 0x91, 0x90, 0x5c, 0x54, 0x41, 0xe3, + 0x65, 0x06, 0x9d, 0x4c, 0x2e, 0x03, 0xf2, 0x47, 0xfb, 0x81, 0x67, 0x98, 0x41, 0x68, 0xbe, 0xba, + 0x70, 0xfe, 0xaa, 0x44, 0x4f, 0x44, 0xbc, 0x3b, 0x29, 0x40, 0x53, 0xcc, 0xf7, 0x0a, 0x93, 0x79, + 0x67, 0x8e, 0x40, 0x66, 0xe6, 0x71, 0xad, 0x19, 0x3e, 0xbf, 0xd8, 0xd9, 0xf3, 0xfb, 0xe1, 0x24, + 0xeb, 0x23, 0x58, 0xee, 0xec, 0x75, 0xda, 0xb5, 0x66, 0x55, 0xc7, 0xb5, 0xce, 0xde, 0x2e, 0xcd, + 0xba, 0x17, 0x60, 0x55, 0xd2, 0x9a, 0x2d, 0x49, 0x56, 0xd0, 0x45, 0x40, 0x92, 0x5c, 0xdf, 0xd1, + 0x77, 0x1a, 0x7b, 0x9d, 0x3b, 0xb5, 0xaa, 0x16, 0x41, 0x2b, 0x90, 0x9e, 0xc0, 0xef, 0x69, 0x51, + 0x91, 0x55, 0x7f, 0x11, 0x81, 0x65, 0xa9, 0xe1, 0xf7, 0xbe, 0x47, 0xa5, 0xf3, 0xee, 0x91, 0x08, + 0xf7, 0xd2, 0xab, 0xae, 0x43, 0xc2, 0x74, 0x07, 0x34, 0x5d, 0x45, 0x17, 0x3a, 0xbf, 0x40, 0xa0, + 0x1b, 0xb4, 0x42, 0x94, 0x87, 0x11, 0x5b, 0x78, 0x18, 0x13, 0x10, 0xbd, 0x2c, 0x81, 0x1b, 0x18, + 0xb6, 0x6e, 0x1e, 0x8e, 0x9c, 0x23, 0x9f, 0x3b, 0x1c, 0x4e, 0x33, 0xda, 0x36, 0x23, 0xa1, 0x57, + 0x60, 0xd9, 0x22, 0xb6, 0x71, 0x42, 0x2c, 0x09, 0x4a, 0x30, 0x50, 0x56, 0x50, 0x39, 0xac, 0xf8, + 0xc7, 0x08, 0x68, 0x58, 0xbc, 0xcb, 0xc8, 0xf7, 0xbf, 0x3c, 0x25, 0xa0, 0x2f, 0xf9, 0xa1, 0xeb, + 0x1b, 0xf6, 0x13, 0x36, 0x3a, 0xc6, 0x9c, 0xdd, 0x6a, 0xf2, 0x69, 0xb6, 0xba, 0x0e, 0x69, 0xc3, + 0x3c, 0x72, 0xdc, 0x4f, 0x6c, 0x62, 0xf5, 0x88, 0x88, 0xb7, 0x61, 0x12, 0xba, 0x05, 0xc8, 0x22, + 0x43, 0x8f, 0xd0, 0x1d, 0x58, 0xfa, 0x13, 0xee, 0xf2, 0xea, 0x04, 0x26, 0x48, 0x8b, 0x5d, 0x92, + 0x46, 0x7a, 0xf1, 0xa9, 0x5b, 0xc4, 0x0e, 0x0c, 0x61, 0xe3, 0x8c, 0x20, 0x56, 0x29, 0xad, 0xf8, + 0x67, 0x05, 0x56, 0x43, 0xd6, 0x7b, 0x86, 0xd1, 0x39, 0x1c, 0x4e, 0xa3, 0x4f, 0x11, 0x4e, 0xbf, + 0xb7, 0x4f, 0x15, 0x7f, 0xad, 0x40, 0xba, 0xd1, 0xf7, 0x03, 0xe9, 0x04, 0xef, 0x82, 0xea, 0x8b, + 0x20, 0x24, 0xdc, 0x60, 0x51, 0x8c, 0x12, 0xae, 0x3f, 0x86, 0xd3, 0xec, 0xfa, 0x89, 0x11, 0x98, + 0x87, 0x32, 0xbb, 0xb2, 0x01, 0xba, 0x09, 0x19, 0xf6, 0xa1, 0x7b, 0xc4, 0x1f, 0x0d, 0x88, 0xc8, + 0x0a, 0xb3, 0x5b, 0x4e, 0x33, 0x14, 0x66, 0xa0, 0xbb, 0x31, 0x35, 0xa2, 0x45, 0xef, 0xc6, 0xd4, + 0xa8, 0x16, 0x2b, 0xfe, 0x2b, 0x02, 0x19, 0xae, 0xe1, 0x33, 0xbf, 0xbe, 0xef, 0x83, 0x2a, 0x1c, + 0x89, 0xbf, 0x35, 0xcf, 0x34, 0x13, 0xc2, 0x3a, 0xc8, 0xb7, 0x88, 0xb4, 0x81, 0x9c, 0x95, 0xff, + 0x83, 0x02, 0xd2, 0xf1, 0xd0, 0x26, 0xc4, 0xe6, 0x17, 0xc4, 0xa1, 0x87, 0x8c, 0x10, 0xc0, 0x80, + 0xf4, 0x7e, 0xd3, 0x82, 0xc0, 0x23, 0x0f, 0xfa, 0xbe, 0xec, 0xab, 0x44, 0x71, 0x7a, 0xe0, 0x5a, + 0x58, 0x90, 0xd0, 0x6b, 0x10, 0xf7, 0xdc, 0x51, 0x40, 0x84, 0x37, 0x84, 0x9a, 0x51, 0x98, 0x92, + 0x85, 0x38, 0x8e, 0x41, 0xff, 0x03, 0x2b, 0xa6, 0x47, 0x8c, 0x80, 0x4c, 0x44, 0xb2, 0xe4, 0x8d, + 0x97, 0x39, 0x59, 0x4a, 0xbd, 0x1b, 0x53, 0x63, 0x5a, 0xbc, 0xf8, 0x37, 0x05, 0x32, 0xe5, 0xe1, + 0xd0, 0x3e, 0x91, 0xbe, 0xf0, 0x1e, 0x24, 0xcd, 0x43, 0xc3, 0xe9, 0x11, 0xd9, 0xfb, 0xba, 0x72, + 0x26, 0xe8, 0x8f, 0x81, 0xa5, 0x6d, 0x86, 0x92, 0x5d, 0x27, 0x31, 0x27, 0xff, 0x85, 0x02, 0x09, + 0xce, 0x41, 0x25, 0x78, 0x8e, 0x1c, 0x0f, 0x89, 0x19, 0xe8, 0x67, 0x36, 0xc8, 0xfa, 0x0d, 0x78, + 0x95, 0xb3, 0x76, 0x43, 0xdb, 0x7c, 0x03, 0x12, 0xa3, 0xa1, 0x4f, 0xbc, 0x40, 0x1c, 0xdc, 0x7c, + 0xe3, 0x61, 0x01, 0x42, 0x57, 0x21, 0x61, 0x11, 0x9b, 0x08, 0xb3, 0x4c, 0xdd, 0x7f, 0xc1, 0x2a, + 0xf6, 0x59, 0xb1, 0x40, 0x95, 0x7e, 0xd6, 0x7e, 0x54, 0xfc, 0x7b, 0x04, 0x34, 0x79, 0x3b, 0xfd, + 0x67, 0x56, 0x9b, 0xcc, 0x56, 0x91, 0xd1, 0xd9, 0x2a, 0x92, 0x56, 0x30, 0xb4, 0x2c, 0x1d, 0x63, + 0xb8, 0x07, 0xd0, 0x52, 0x55, 0x22, 0x5e, 0x85, 0x15, 0x87, 0x1c, 0x07, 0xfa, 0xd0, 0xe8, 0x11, + 0x3d, 0x70, 0x8f, 0x88, 0x23, 0xa2, 0x5e, 0x96, 0x92, 0xdb, 0x46, 0x8f, 0x74, 0x29, 0x11, 0x5d, + 0x01, 0x60, 0x10, 0xfe, 0x1e, 0xa3, 0x21, 0x39, 0x8e, 0x53, 0x94, 0xc2, 0x1e, 0x63, 0xe8, 0x36, + 0x64, 0xfc, 0x7e, 0xcf, 0x31, 0x82, 0x91, 0x47, 0xba, 0xdd, 0x86, 0x88, 0xf3, 0x4f, 0xe8, 0x4b, + 0xa8, 0x0f, 0x4f, 0x0b, 0x0a, 0x6b, 0x3a, 0x9c, 0x99, 0x38, 0x53, 0x73, 0xa9, 0xd3, 0x35, 0x57, + 0xf1, 0xeb, 0x08, 0xac, 0x86, 0xec, 0xfb, 0xcc, 0xe3, 0x42, 0x1d, 0x52, 0x32, 0xc4, 0xca, 0xc0, + 0xf0, 0xca, 0x6c, 0x1c, 0x1e, 0x6b, 0x52, 0xd2, 0xc7, 0xcd, 0x46, 0x2e, 0x67, 0x32, 0x7b, 0x9e, + 0xb1, 0x63, 0x73, 0x8c, 0x9d, 0xff, 0x08, 0x52, 0x63, 0x29, 0xe8, 0xf5, 0x33, 0x91, 0x64, 0x4e, + 0x0a, 0x38, 0x13, 0x46, 0xae, 0x00, 0x50, 0x7b, 0x12, 0x8b, 0x55, 0xd4, 0xfc, 0x1d, 0x9f, 0xe2, + 0x94, 0x3d, 0xcf, 0x2e, 0xfe, 0x4c, 0x81, 0x38, 0x0b, 0x16, 0xe8, 0x1d, 0x48, 0x0e, 0xc8, 0x60, + 0x9f, 0xe6, 0x0a, 0x7e, 0xbf, 0xcf, 0xeb, 0x32, 0x48, 0x38, 0x4d, 0xa0, 0x43, 0xaf, 0x3f, 0x30, + 0xbc, 0x13, 0xde, 0xfc, 0xc5, 0x72, 0x88, 0xae, 0x43, 0x4a, 0xb6, 0x19, 0x64, 0xbf, 0xee, 0x6c, + 0x17, 0x62, 0xc2, 0x16, 0x05, 0xda, 0xef, 0x22, 0x90, 0xe0, 0x56, 0x47, 0xef, 0x01, 0xc8, 0x56, + 0xc2, 0x53, 0x77, 0x3e, 0x52, 0x62, 0x46, 0xdd, 0x9a, 0x04, 0xc7, 0xc8, 0x53, 0x04, 0xc7, 0x4d, + 0x88, 0x91, 0xc0, 0xb4, 0x44, 0x20, 0xbd, 0x30, 0xed, 0x01, 0xa5, 0x5a, 0x60, 0x5a, 0xd2, 0xac, + 0x14, 0x98, 0xff, 0x4c, 0x81, 0x18, 0x25, 0x52, 0xfb, 0x9a, 0xf6, 0x88, 0xa6, 0x4f, 0xa9, 0x65, + 0x0c, 0xa7, 0x04, 0xa5, 0x6e, 0xa1, 0x17, 0x20, 0xc5, 0xcd, 0x44, 0xb9, 0x11, 0xc6, 0x55, 0x39, + 0xa1, 0x6e, 0xa1, 0x3c, 0xa8, 0xe3, 0xe8, 0xc7, 0x6f, 0xeb, 0x78, 0x4c, 0x27, 0x7a, 0xc6, 0x41, + 0xa0, 0x07, 0xc4, 0xe3, 0xfd, 0x85, 0x18, 0x56, 0x29, 0xa1, 0x4b, 0xbc, 0x81, 0x6c, 0xc0, 0xd0, + 0xdf, 0xeb, 0xff, 0x89, 0x40, 0x82, 0x7b, 0x34, 0x4a, 0x40, 0xa4, 0xf5, 0x01, 0xaf, 0x95, 0xef, + 0xb6, 0xf6, 0x70, 0xb3, 0xdc, 0xd0, 0x9b, 0xad, 0xae, 0xbe, 0xd3, 0xda, 0x6b, 0x56, 0x35, 0x05, + 0x5d, 0x81, 0xcb, 0xcd, 0x96, 0x2e, 0x39, 0x6d, 0x5c, 0xdf, 0x2d, 0xe3, 0xfb, 0x7a, 0x05, 0xb7, + 0x3e, 0xa8, 0x61, 0x2d, 0x82, 0xd6, 0x20, 0x4f, 0xd1, 0x0b, 0xf8, 0x51, 0x5a, 0x6a, 0x87, 0xf9, + 0x82, 0x1e, 0x47, 0xeb, 0xf0, 0x62, 0xbd, 0xd9, 0xd9, 0xdb, 0xd9, 0xa9, 0x6f, 0xd7, 0x6b, 0xcd, + 0x69, 0x40, 0x47, 0x8b, 0xa1, 0x17, 0x21, 0xd7, 0xda, 0xd9, 0xe9, 0xd4, 0xba, 0x4c, 0x9d, 0xfb, + 0xb5, 0xae, 0x5e, 0xfe, 0xb0, 0x5c, 0x6f, 0x94, 0x2b, 0x8d, 0x9a, 0x96, 0xa0, 0xa5, 0xfa, 0x3d, + 0xdc, 0x6a, 0xde, 0xd6, 0x71, 0x6b, 0xaf, 0x5b, 0xd3, 0x92, 0x54, 0xfd, 0x36, 0x6e, 0xb5, 0x5b, + 0x9d, 0x72, 0x43, 0xdf, 0xad, 0x77, 0x76, 0xcb, 0xdd, 0xed, 0x3b, 0x9a, 0x8a, 0x5e, 0x80, 0x4b, + 0xb5, 0xee, 0x76, 0x55, 0xef, 0xe2, 0x72, 0xb3, 0x53, 0xde, 0xee, 0xd6, 0x5b, 0x4d, 0x7d, 0xa7, + 0x5c, 0x6f, 0xd4, 0xaa, 0x5a, 0x8a, 0x0a, 0xa1, 0xb2, 0xcb, 0x8d, 0x46, 0xeb, 0x5e, 0xad, 0xaa, + 0x01, 0xba, 0x04, 0xcf, 0x71, 0xa9, 0xe5, 0x36, 0x7b, 0x05, 0x70, 0x05, 0xb4, 0x34, 0x55, 0xa6, + 0xde, 0xac, 0xd6, 0x3e, 0xd2, 0xef, 0x94, 0x3b, 0xfa, 0x6d, 0x5c, 0x2b, 0x77, 0x6b, 0x58, 0x72, + 0x33, 0x74, 0x6d, 0x5c, 0xbb, 0x5d, 0xef, 0x50, 0xe2, 0x78, 0xed, 0x2c, 0xca, 0x42, 0x4a, 0x3c, + 0x27, 0x6a, 0x55, 0x6d, 0xf9, 0xba, 0x03, 0xda, 0x74, 0xa3, 0x06, 0xa5, 0x21, 0x59, 0x6f, 0x7e, + 0x58, 0x6e, 0xd4, 0xab, 0xda, 0xd2, 0xb8, 0x5b, 0xc8, 0x1a, 0x84, 0xb7, 0x3f, 0xae, 0xb7, 0xb5, + 0x08, 0x95, 0xf1, 0x71, 0xa7, 0x5b, 0x6e, 0x56, 0xcb, 0xb8, 0xaa, 0x45, 0x11, 0x40, 0xa2, 0xd3, + 0x2c, 0xb7, 0xdb, 0xf7, 0xb5, 0x18, 0x35, 0x3d, 0x05, 0x51, 0x35, 0x1a, 0xad, 0x72, 0x55, 0xaf, + 0xd6, 0xb6, 0x5b, 0xbb, 0x6d, 0x5c, 0xeb, 0x74, 0xea, 0xad, 0xa6, 0x16, 0xdf, 0xfa, 0x3c, 0x3a, + 0x29, 0x24, 0xde, 0x86, 0x18, 0x2d, 0x3e, 0xd0, 0x85, 0xe9, 0x62, 0x84, 0x25, 0x96, 0xfc, 0xc5, + 0xf9, 0x35, 0xca, 0x0d, 0x05, 0xbd, 0x03, 0x71, 0x96, 0xf2, 0xd0, 0xc5, 0xf9, 0x89, 0x3b, 0x7f, + 0x69, 0x86, 0x2e, 0x62, 0xe9, 0xdb, 0x10, 0xc3, 0xf4, 0x3d, 0x1f, 0x5a, 0x32, 0xd4, 0xbd, 0x09, + 0x2f, 0x19, 0xee, 0x50, 0xdc, 0x50, 0xd0, 0x7b, 0x90, 0xe0, 0xaf, 0x2d, 0x74, 0x69, 0xc1, 0x0b, + 0x31, 0x9f, 0x9b, 0x65, 0xf0, 0xe9, 0x1b, 0x0a, 0xba, 0x03, 0xa9, 0x71, 0x65, 0x8d, 0xf2, 0xe1, + 0x55, 0xce, 0x3e, 0x56, 0xf2, 0x2f, 0xcc, 0xe5, 0x49, 0x39, 0x37, 0xa8, 0xa4, 0x2c, 0xb5, 0xc6, + 0x38, 0x38, 0x87, 0xa5, 0x4d, 0xe7, 0xe6, 0xb0, 0xb4, 0x99, 0x68, 0x5e, 0xa9, 0x3d, 0xfc, 0xe7, + 0xda, 0xd2, 0xc3, 0x6f, 0xd7, 0x94, 0x6f, 0xbe, 0x5d, 0x53, 0x7e, 0xfe, 0x68, 0x6d, 0xe9, 0xab, + 0x47, 0x6b, 0xca, 0x9f, 0x1e, 0xad, 0x29, 0xdf, 0x3c, 0x5a, 0x5b, 0xfa, 0xcb, 0xa3, 0xb5, 0xa5, + 0x8f, 0xaf, 0xf6, 0xdc, 0x52, 0xcf, 0xf8, 0x94, 0x04, 0x01, 0x29, 0x59, 0xe4, 0xc1, 0xa6, 0xe9, + 0x7a, 0x64, 0x73, 0xea, 0xbf, 0xd0, 0xfd, 0x04, 0xfb, 0xba, 0xf9, 0xdf, 0x00, 0x00, 0x00, 0xff, + 0xff, 0x78, 0x0f, 0xa4, 0x40, 0x25, 0x1d, 0x00, 0x00, } func (this *Label) Equal(that interface{}) bool { @@ -2069,6 +2226,9 @@ func (this *JournalSpec) Equal(that interface{}) bool { if this.MaxAppendRate != that1.MaxAppendRate { return false } + if !this.Suspend.Equal(that1.Suspend) { + return false + } return true } func (this *JournalSpec_Fragment) Equal(that interface{}) bool { @@ -2118,6 +2278,33 @@ func (this *JournalSpec_Fragment) Equal(that interface{}) bool { } return true } +func (this *JournalSpec_Suspend) Equal(that interface{}) bool { + if that == nil { + return this == nil + } + + that1, ok := that.(*JournalSpec_Suspend) + if !ok { + that2, ok := that.(JournalSpec_Suspend) + if ok { + that1 = &that2 + } else { + return false + } + } + if that1 == nil { + return this == nil + } else if this == nil { + return false + } + if this.Level != that1.Level { + return false + } + if this.Offset != that1.Offset { + return false + } + return true +} func (this *ProcessSpec_ID) Equal(that interface{}) bool { if that == nil { return this == nil @@ -2185,6 +2372,9 @@ func (this *AppendRequest) Equal(that interface{}) bool { if !this.SubtractRegisters.Equal(that1.SubtractRegisters) { return false } + if this.Suspend != that1.Suspend { + return false + } if !bytes.Equal(this.Content, that1.Content) { return false } @@ -2838,6 +3028,18 @@ func (m *JournalSpec) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Suspend != nil { + { + size, err := m.Suspend.MarshalToSizedBuffer(dAtA[:i]) + if err != nil { + return 0, err + } + i -= size + i = encodeVarintProtocol(dAtA, i, uint64(size)) + } + i-- + dAtA[i] = 0x42 + } if m.MaxAppendRate != 0 { i = encodeVarintProtocol(dAtA, i, uint64(m.MaxAppendRate)) i-- @@ -2910,29 +3112,29 @@ func (m *JournalSpec_Fragment) MarshalToSizedBuffer(dAtA []byte) (int, error) { i-- dAtA[i] = 0x3a } - n5, err5 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.FlushInterval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.FlushInterval):]) - if err5 != nil { - return 0, err5 - } - i -= n5 - i = encodeVarintProtocol(dAtA, i, uint64(n5)) - i-- - dAtA[i] = 0x32 - n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Retention, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Retention):]) + n6, err6 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.FlushInterval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.FlushInterval):]) if err6 != nil { return 0, err6 } i -= n6 i = encodeVarintProtocol(dAtA, i, uint64(n6)) i-- - dAtA[i] = 0x2a - n7, err7 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.RefreshInterval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.RefreshInterval):]) + dAtA[i] = 0x32 + n7, err7 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.Retention, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.Retention):]) if err7 != nil { return 0, err7 } i -= n7 i = encodeVarintProtocol(dAtA, i, uint64(n7)) i-- + dAtA[i] = 0x2a + n8, err8 := github_com_gogo_protobuf_types.StdDurationMarshalTo(m.RefreshInterval, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(m.RefreshInterval):]) + if err8 != nil { + return 0, err8 + } + i -= n8 + i = encodeVarintProtocol(dAtA, i, uint64(n8)) + i-- dAtA[i] = 0x22 if len(m.Stores) > 0 { for iNdEx := len(m.Stores) - 1; iNdEx >= 0; iNdEx-- { @@ -2956,6 +3158,39 @@ func (m *JournalSpec_Fragment) MarshalToSizedBuffer(dAtA []byte) (int, error) { return len(dAtA) - i, nil } +func (m *JournalSpec_Suspend) Marshal() (dAtA []byte, err error) { + size := m.ProtoSize() + dAtA = make([]byte, size) + n, err := m.MarshalToSizedBuffer(dAtA[:size]) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *JournalSpec_Suspend) MarshalTo(dAtA []byte) (int, error) { + size := m.ProtoSize() + return m.MarshalToSizedBuffer(dAtA[:size]) +} + +func (m *JournalSpec_Suspend) MarshalToSizedBuffer(dAtA []byte) (int, error) { + i := len(dAtA) + _ = i + var l int + _ = l + if m.Offset != 0 { + i = encodeVarintProtocol(dAtA, i, uint64(m.Offset)) + i-- + dAtA[i] = 0x10 + } + if m.Level != 0 { + i = encodeVarintProtocol(dAtA, i, uint64(m.Level)) + i-- + dAtA[i] = 0x8 + } + return len(dAtA) - i, nil +} + func (m *ProcessSpec) Marshal() (dAtA []byte, err error) { size := m.ProtoSize() dAtA = make([]byte, size) @@ -3369,6 +3604,11 @@ func (m *AppendRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { _ = i var l int _ = l + if m.Suspend != 0 { + i = encodeVarintProtocol(dAtA, i, uint64(m.Suspend)) + i-- + dAtA[i] = 0x48 + } if m.SubtractRegisters != nil { { size, err := m.SubtractRegisters.MarshalToSizedBuffer(dAtA[:i]) @@ -3986,12 +4226,12 @@ func (m *FragmentsRequest) MarshalToSizedBuffer(dAtA []byte) (int, error) { dAtA[i] = 0x40 } if m.SignatureTTL != nil { - n34, err34 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.SignatureTTL, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.SignatureTTL):]) - if err34 != nil { - return 0, err34 + n35, err35 := github_com_gogo_protobuf_types.StdDurationMarshalTo(*m.SignatureTTL, dAtA[i-github_com_gogo_protobuf_types.SizeOfStdDuration(*m.SignatureTTL):]) + if err35 != nil { + return 0, err35 } - i -= n34 - i = encodeVarintProtocol(dAtA, i, uint64(n34)) + i -= n35 + i = encodeVarintProtocol(dAtA, i, uint64(n35)) i-- dAtA[i] = 0x3a } @@ -4363,6 +4603,10 @@ func (m *JournalSpec) ProtoSize() (n int) { if m.MaxAppendRate != 0 { n += 1 + sovProtocol(uint64(m.MaxAppendRate)) } + if m.Suspend != nil { + l = m.Suspend.ProtoSize() + n += 1 + l + sovProtocol(uint64(l)) + } return n } @@ -4397,6 +4641,21 @@ func (m *JournalSpec_Fragment) ProtoSize() (n int) { return n } +func (m *JournalSpec_Suspend) ProtoSize() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.Level != 0 { + n += 1 + sovProtocol(uint64(m.Level)) + } + if m.Offset != 0 { + n += 1 + sovProtocol(uint64(m.Offset)) + } + return n +} + func (m *ProcessSpec) ProtoSize() (n int) { if m == nil { return 0 @@ -4601,6 +4860,9 @@ func (m *AppendRequest) ProtoSize() (n int) { l = m.SubtractRegisters.ProtoSize() n += 1 + l + sovProtocol(uint64(l)) } + if m.Suspend != 0 { + n += 1 + sovProtocol(uint64(m.Suspend)) + } return n } @@ -5455,6 +5717,42 @@ func (m *JournalSpec) Unmarshal(dAtA []byte) error { break } } + case 8: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Suspend", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= int(b&0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthProtocol + } + postIndex := iNdEx + msglen + if postIndex < 0 { + return ErrInvalidLengthProtocol + } + if postIndex > l { + return io.ErrUnexpectedEOF + } + if m.Suspend == nil { + m.Suspend = &JournalSpec_Suspend{} + } + if err := m.Suspend.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex default: iNdEx = preIndex skippy, err := skipProtocol(dAtA[iNdEx:]) @@ -5727,6 +6025,94 @@ func (m *JournalSpec_Fragment) Unmarshal(dAtA []byte) error { } return nil } +func (m *JournalSpec_Suspend) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= uint64(b&0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: Suspend: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: Suspend: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Level", wireType) + } + m.Level = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Level |= JournalSpec_Suspend_Level(b&0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Offset", wireType) + } + m.Offset = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Offset |= int64(b&0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipProtocol(dAtA[iNdEx:]) + if err != nil { + return err + } + if (skippy < 0) || (iNdEx+skippy) < 0 { + return ErrInvalidLengthProtocol + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} func (m *ProcessSpec) Unmarshal(dAtA []byte) error { l := len(dAtA) iNdEx := 0 @@ -7151,6 +7537,25 @@ func (m *AppendRequest) Unmarshal(dAtA []byte) error { return err } iNdEx = postIndex + case 9: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Suspend", wireType) + } + m.Suspend = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowProtocol + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Suspend |= AppendRequest_Suspend(b&0x7F) << shift + if b < 0x80 { + break + } + } default: iNdEx = preIndex skippy, err := skipProtocol(dAtA[iNdEx:]) diff --git a/broker/protocol/protocol.proto b/broker/protocol/protocol.proto index 582b4327..2ca377dd 100644 --- a/broker/protocol/protocol.proto +++ b/broker/protocol/protocol.proto @@ -58,6 +58,11 @@ enum Status { // The Append is refused because a registers selector was provided with the // request, but it was not matched by current register values of the journal. REGISTER_MISMATCH = 13; + // The operation cannot complete because the journal has been suspended. + // This implicitly means the journal has no available content to read. + // An Append will resume the journal, or the client may want to filter + // further requests directed to suspended journals. + SUSPENDED = 14; } // CompressionCode defines codecs known to Gazette. @@ -288,6 +293,44 @@ message JournalSpec { // smaller of the journal vs global rate. int64 max_append_rate = 7 [ (gogoproto.moretags) = "yaml:\"max_append_rate,omitempty\"" ]; + + // Suspension control for this journal. + // If unset the suspension level is implicitly NONE, with an offset of zero. + // + // The Suspend field is managed by Gazette, and is updated when an Append RPC + // suspends a journal or if auto-suspension is enabled. Operators should + // not set it directly. However when utilizing suspension, operators MUST + // take care to pass-through Suspend when applying updates to JournalSpecs. + message Suspend { + option (gogoproto.equal) = true; + + enum Level { + // When NONE, the journal is not suspended and is fully replicated. + NONE = 0x00; + // When PARTIAL, the journal is scaled down to a single replica + // which monitors the fragment index and serves reads. + PARTIAL = 0x01; + // When FULL, the journal's fragment index MUST be empty and the journal + // is scaled down to zero replicas. Operations other than Append, + // which resumes the journal, will fail with status SUSPENDED. + // + // Clients should identify and filter requests which would otherwise be + // directed at suspended journals. For example, a client may watch a + // journal listing and filter to read journals which have a suspension + // level of ACTIVE or PARTIAL. + FULL = 0x02; + } + Level level = 1 + [ (gogoproto.moretags) = "yaml:\",omitempty\"" ]; + + // The lower-bound journal offset at which appends should proceed, + // once this journal is resumed. + int64 offset = 2 + [ (gogoproto.moretags) = "yaml:\",omitempty\"" ]; + } + Suspend suspend = 8 [ + (gogoproto.moretags) = "yaml:\",omitempty\"" + ]; } // ProcessSpec describes a uniquely identified process and its addressable @@ -496,6 +539,28 @@ message AppendRequest { // Labels to subtract from current registers if the RPC succeeds and appended // at least one byte. LabelSet subtract_registers = 8; + // Suspension control for this request. + enum Suspend { + // SUSPEND_RESUME (the default) will resume a suspended journal. + SUSPEND_RESUME = 0x00; + // SUSPEND_NO_RESUME fails with status SUSPENDED if the journal is suspended. + SUSPEND_NO_RESUME = 0x01; + // SUSPEND_IF_FLUSHED potentially suspends the requested journal: + // - If the journal has local content which is not yet available in the + // remote fragment store, the operation has no effect. + // - If the journal has not content at all, it is fully suspended. + // - Otherwise, the journal is partially suspended. + SUSPEND_IF_FLUSHED = 0x02; + // SUSPEND_NOW is similar to SUSPEND_IF_FLUSHED, but will also + // partially suspend the journal even if it has local content which + // is not yet available in the remote fragment store. + // + // This operation is always safe -- even once suspended, all former replicas + // persist content to the remote store as per usual -- but it can result in + // many small files if a journal is repeatedly suspended and resumed. + SUSPEND_NOW = 0x03; + } + Suspend suspend = 9; // Content chunks to be appended. Immediately prior to closing the stream, // the client must send an empty chunk (eg, zero-valued AppendRequest) to diff --git a/broker/protocol/rpc_extensions.go b/broker/protocol/rpc_extensions.go index 255d3e45..5c527abd 100644 --- a/broker/protocol/rpc_extensions.go +++ b/broker/protocol/rpc_extensions.go @@ -122,6 +122,15 @@ func (m *ReadResponse) Validate() error { return nil } +func (x AppendRequest_Suspend) Validate() error { + switch x { + case AppendRequest_SUSPEND_RESUME, AppendRequest_SUSPEND_NO_RESUME, AppendRequest_SUSPEND_IF_FLUSHED, AppendRequest_SUSPEND_NOW: + return nil + default: + return NewValidationError("invalid Suspend variant (%s)", x) + } +} + // Validate returns an error if the AppendRequest is not well-formed. func (m *AppendRequest) Validate() error { if m.Journal != "" { @@ -152,6 +161,9 @@ func (m *AppendRequest) Validate() error { return ExtendContext(err, "SubtractRegisters") } } + if err := m.Suspend.Validate(); err != nil { + return ExtendContext(err, "Suspend") + } } else if m.Header != nil { return NewValidationError("unexpected Header") } else if m.DoNotProxy { @@ -164,6 +176,8 @@ func (m *AppendRequest) Validate() error { return NewValidationError("unexpected UnionRegisters") } else if m.SubtractRegisters != nil { return NewValidationError("unexpected SubtractRegisters") + } else if m.Suspend != AppendRequest_SUSPEND_RESUME { + return NewValidationError("unexpected Suspend (%v; expected SUSPEND_RESUMED)", m.Suspend) } return nil } diff --git a/broker/protocol/rpc_extensions_test.go b/broker/protocol/rpc_extensions_test.go index a9bcf228..866b38c4 100644 --- a/broker/protocol/rpc_extensions_test.go +++ b/broker/protocol/rpc_extensions_test.go @@ -126,6 +126,7 @@ func (s *RPCSuite) TestAppendRequestValidationCases(c *gc.C) { CheckRegisters: &LabelSelector{Include: badLabel}, UnionRegisters: &badLabel, SubtractRegisters: &badLabel, + Suspend: 12345, } c.Check(req.Validate(), gc.ErrorMatches, `Header.Etcd: invalid ClusterId .*`) @@ -142,6 +143,8 @@ func (s *RPCSuite) TestAppendRequestValidationCases(c *gc.C) { req.UnionRegisters = &goodLabel c.Check(req.Validate(), gc.ErrorMatches, `SubtractRegisters.Labels\[0\].Name: not a valid token \(inv alid\)`) req.SubtractRegisters = &goodLabel + c.Check(req.Validate(), gc.ErrorMatches, `Suspend: invalid Suspend variant \(12345\)`) + req.Suspend = AppendRequest_SUSPEND_NOW c.Check(req.Validate(), gc.IsNil) @@ -161,6 +164,8 @@ func (s *RPCSuite) TestAppendRequestValidationCases(c *gc.C) { req.UnionRegisters = nil c.Check(req.Validate(), gc.ErrorMatches, `unexpected SubtractRegisters`) req.SubtractRegisters = nil + c.Check(req.Validate(), gc.ErrorMatches, `unexpected Suspend \(SUSPEND_NOW; expected SUSPEND_RESUMED\)`) + req.Suspend = AppendRequest_SUSPEND_RESUME c.Check(req.Validate(), gc.IsNil) diff --git a/broker/replica.go b/broker/replica.go index 006944fd..c32e1985 100644 --- a/broker/replica.go +++ b/broker/replica.go @@ -15,6 +15,9 @@ import ( "go.gazette.dev/core/keyspace" ) +// AutoSuspend journals which have no local fragments. +var AutoSuspend bool = false + // replica is a runtime instance of a journal which is assigned to this broker. type replica struct { journal pb.Journal @@ -103,21 +106,26 @@ func fragmentRefreshDaemon(ks *keyspace.KeySpace, r *replica) { // on changes to the replica Route. Additional periodic pulses ensure problems // with the peer set (eg, half-broken connections) are detected proactively. func pulseDaemon(svc *Service, r *replica) { - var timer = time.NewTimer(0) // Fires immediately. + var ( + invalidateCh <-chan struct{} // Signaled upon routing changes. + stableTicks int // Number of health checks since the last routing change. + timer = time.NewTimer(0) // Health-check interval timer. + ) + <-timer.C // Dequeue first fire. defer timer.Stop() - var invalidateCh <-chan struct{} for { - select { - case <-r.ctx.Done(): - return - case <-timer.C: - timer.Reset(healthCheckInterval) - case <-invalidateCh: - invalidateCh = nil + var ctx, cancel = context.WithTimeout(r.ctx, healthCheckInterval) + + // If AutoSuspend is enabled, use the SUSPEND_IF_FLUSHED control flag + // after at least one `healthCheckInterval` has elapsed. We cannot use + // SUSPEND_IF_FLUSHED immediately because there's a natural startup race + // with the client(s) which resumed and intend to append to the journal. + var suspend = pb.AppendRequest_SUSPEND_NO_RESUME + if AutoSuspend && stableTicks >= 1 { + suspend = pb.AppendRequest_SUSPEND_IF_FLUSHED } - var ctx, cancel = context.WithTimeout(r.ctx, healthCheckInterval) var fsm = appendFSM{ svc: svc, ctx: ctx, @@ -130,6 +138,7 @@ func pulseDaemon(svc *Service, r *replica) { req: pb.AppendRequest{ Journal: r.journal, DoNotProxy: true, + Suspend: suspend, }, } if fsm.runTo(stateStreamContent) { @@ -146,6 +155,8 @@ func pulseDaemon(svc *Service, r *replica) { // Only the primary pulses the journal. No-op. } else if fsm.resolved.status == pb.Status_JOURNAL_NOT_FOUND { // Journal was deleted while we waited. + } else if fsm.resolved.status == pb.Status_SUSPENDED { + // Journal was suspended. } else if errors.Cause(fsm.err) == context.Canceled { // Replica is shutting down. } else if errors.Cause(fsm.err) == errResolverStopped { @@ -157,11 +168,24 @@ func pulseDaemon(svc *Service, r *replica) { "journal": r.journal, }).Warn("journal pulse failed (will retry)") } + cancel() + // Loop again if the current topology changes. if fsm.resolved != nil { invalidateCh = fsm.resolved.invalidateCh } - cancel() + // Or if `healthCheckInterval` elapses. + timer.Reset(healthCheckInterval) + + select { + case <-r.ctx.Done(): + return + case <-timer.C: + stableTicks += 1 + case <-invalidateCh: + invalidateCh = nil + stableTicks = 0 + } } } @@ -225,6 +249,40 @@ func updateAssignments(ctx context.Context, assignments keyspace.KeyValues, etcd } } +func updateJournalSpec( + ctx context.Context, + item keyspace.KeyValue, + assignments keyspace.KeyValues, + update pb.JournalSpec, + etcd clientv3.KV, +) (int64, error) { + + // Construct an Etcd transaction which asserts `item` and `assignments` are + // unchanged, and which updates `item` to have value `update`. + var cmp []clientv3.Cmp + var ops []clientv3.Op + var itemKey = string(item.Raw.Key) + + cmp = append(cmp, clientv3.Compare(clientv3.ModRevision(itemKey), "=", item.Raw.ModRevision)) + ops = append(ops, clientv3.OpPut(itemKey, update.MarshalString())) + + for _, kv := range assignments { + var key = string(kv.Raw.Key) + cmp = append(cmp, clientv3.Compare(clientv3.ModRevision(key), "=", kv.Raw.ModRevision)) + } + + // Attempt a transaction which may win or lose its race. + // The caller reads through the returned revision to determine the outcome. + if resp, err := etcd.Txn(ctx).If(cmp...).Then(ops...).Commit(); err != nil { + if ctx.Err() != nil { + err = ctx.Err() + } + return 0, err + } else { + return resp.Header.Revision, nil + } +} + // maybeRollFragment returns either the current Fragment, or an empty Fragment // which has been "rolled" to an offset at or after the current Spool End, // and which reflects the latest |spec|. diff --git a/broker/replicate_api_test.go b/broker/replicate_api_test.go index 4cd779ae..637a3f21 100644 --- a/broker/replicate_api_test.go +++ b/broker/replicate_api_test.go @@ -40,7 +40,7 @@ func TestReplicateStreamAndCommit(t *testing.T) { require.NoError(t, stream.Send(&pb.ReplicateRequest{Content: []byte("bazbing"), ContentDelta: 6})) // Precondition: content not observable in the Fragment index. - var _, eo = broker.replica("a/journal").index.OffsetRange() + var _, eo, _ = broker.replica("a/journal").index.OffsetRange() require.Equal(t, int64(0), eo) // Commit. @@ -58,7 +58,7 @@ func TestReplicateStreamAndCommit(t *testing.T) { expectReplResponse(t, stream, &pb.ReplicateResponse{Status: pb.Status_OK}) // Post-condition: content is now observable. - _, eo = broker.replica("a/journal").index.OffsetRange() + _, eo, _ = broker.replica("a/journal").index.OffsetRange() require.Equal(t, int64(13), eo) // Send EOF and expect its returned. diff --git a/broker/resolver.go b/broker/resolver.go index 7a70d836..b4b4aadf 100644 --- a/broker/resolver.go +++ b/broker/resolver.go @@ -66,7 +66,9 @@ type resolution struct { // resolution is proxy-able to multiple peers, but is always specified if this // broker can locally serve the request, or the primary broker is required. pb.Header - // JournalSpec of the Journal at the current Etcd Revision. + // Item (JournalSpec) of the Journal at the current Etcd Revision. + item keyspace.KeyValue + // Decoded JournalSpec extracted from `item`. journalSpec *pb.JournalSpec // Assignments of the Journal at the current Etcd Revision. assignments keyspace.KeyValues @@ -135,11 +137,15 @@ func (r *resolver) resolve(ctx context.Context, claims pb.Claims, journal pb.Jou allocator.ItemAssignmentsPrefix(ks, journal.String())).Copy() // Extract JournalSpec. - if item, ok := allocator.LookupItem(ks, journal.String()); ok { - var spec = item.ItemValue.(*pb.JournalSpec) + if index, ok := ks.KeyValues.Search(allocator.ItemKey(ks, journal.String())); ok { + var item = ks.KeyValues[index] + var spec = item.Decoded.(allocator.Item).ItemValue.(*pb.JournalSpec) // Is the caller authorized to the journal? - if claims.Selector.Matches(spec.LabelSetExt(pb.LabelSet{})) { + if claims.Selector.Matches(spec.LabelSetExt(pb.LabelSet{ + Labels: make([]pb.Label, 0, 1+len(spec.LabelSet.Labels)), + })) { + res.item = item res.journalSpec = spec } else { // Clear to act as if the journal doesn't exist. @@ -182,6 +188,8 @@ func (r *resolver) resolve(ctx context.Context, claims pb.Claims, journal pb.Jou // Select a response Status code. if res.journalSpec == nil { res.status = pb.Status_JOURNAL_NOT_FOUND + } else if res.journalSpec.Suspend.GetLevel() == pb.JournalSpec_Suspend_FULL { + res.status = pb.Status_SUSPENDED } else if opts.requirePrimary && res.Route.Primary == -1 { res.status = pb.Status_NO_JOURNAL_PRIMARY_BROKER } else if len(res.Route.Members) == 0 { diff --git a/cmd/gazctl/gazctlcmd/journals_fragments.go b/cmd/gazctl/gazctlcmd/journals_fragments.go index c91ae4ff..54115c79 100644 --- a/cmd/gazctl/gazctlcmd/journals_fragments.go +++ b/cmd/gazctl/gazctlcmd/journals_fragments.go @@ -97,6 +97,9 @@ func (cmd *cmdJournalsFragments) Execute([]string) error { var responses = make([]*pb.FragmentsResponse, len(listResponse.Journals)) for i := range listResponse.Journals { + if listResponse.Journals[i].Spec.Suspend.GetLevel() == pb.JournalSpec_Suspend_FULL { + continue + } wg.Add(1) go func(i int) { @@ -145,6 +148,9 @@ func (cmd *cmdJournalsFragments) outputTable(responses []*pb.FragmentsResponse) table.SetHeader(headers) for _, r := range responses { + if r == nil { + continue // Skipped due to suspension. + } for _, f := range r.Fragments { var sum = f.Spec.Sum.ToDigest() var modTime string diff --git a/cmd/gazctl/gazctlcmd/journals_list.go b/cmd/gazctl/gazctlcmd/journals_list.go index dc4adbab..cd5f46a0 100644 --- a/cmd/gazctl/gazctlcmd/journals_list.go +++ b/cmd/gazctl/gazctlcmd/journals_list.go @@ -70,8 +70,8 @@ func (cmd *cmdJournalsList) Execute([]string) error { mbp.Must(<-list.UpdateCh(), "failed to list journals") for { - <-list.UpdateCh() cmd.output(list.List()) + <-list.UpdateCh() } } @@ -105,9 +105,7 @@ func (cmd *cmdJournalsList) outputTable(resp *pb.ListResponse) { if cmd.Stores { headers = append(headers, "Stores") } - for _, l := range cmd.Labels { - headers = append(headers, l) - } + headers = append(headers, cmd.Labels...) table.SetHeader(headers) for _, j := range resp.Journals { @@ -122,6 +120,10 @@ func (cmd *cmdJournalsList) outputTable(resp *pb.ListResponse) { } } + if j.Spec.Suspend.GetLevel() != pb.JournalSpec_Suspend_NONE { + primary = primary + " " + } + var row = []string{ j.Spec.Name.String(), } diff --git a/cmd/gazctl/gazctlcmd/journals_read.go b/cmd/gazctl/gazctlcmd/journals_read.go index 910dbe55..203eaccc 100644 --- a/cmd/gazctl/gazctlcmd/journals_read.go +++ b/cmd/gazctl/gazctlcmd/journals_read.go @@ -24,12 +24,11 @@ type cmdJournalRead struct { FileRoot string `long:"file-root" description:"Filesystem path which roots file:// fragment store"` FromUnix int64 `long:"from" description:"Skip over fragments persisted before this time, in unix seconds since epoch"` - pumpCh chan pumpResult // Chan into which completed read pumps are sent. - beginOffsets map[pb.Journal]int64 // Contents of initial --offsets. - endOffsets map[pb.Journal]int64 // Collected --offsets-out. - cancelFns map[pb.Journal]context.CancelFunc // CancelFuncs of active read pumps. - output *os.File // Output to which we're multiplexing reads. - buffer []byte // Buffer for copying to |output| + pumpCh chan pumpResult // Chan into which completed read pumps are sent. + offsets map[pb.Journal]int64 // Journal read offsets. + cancelFns map[pb.Journal]context.CancelFunc // CancelFuncs of active read pumps. + output *os.File // Output to which we're multiplexing reads. + buffer []byte // Buffer for copying to |output| } func init() { @@ -99,14 +98,13 @@ func (cmd *cmdJournalRead) Execute([]string) error { client.InstallFileTransport(cmd.FileRoot) } cmd.pumpCh = make(chan pumpResult) - cmd.beginOffsets = make(map[pb.Journal]int64) - cmd.endOffsets = make(map[pb.Journal]int64) + cmd.offsets = make(map[pb.Journal]int64) cmd.buffer = make([]byte, 32*1024) if cmd.OffsetsPath != "" { var fin, err = os.Open(cmd.OffsetsPath) mbp.Must(err, "failed to open offsets for reading") - mbp.Must(json.NewDecoder(fin).Decode(&cmd.beginOffsets), "failed to decode offsets") + mbp.Must(json.NewDecoder(fin).Decode(&cmd.offsets), "failed to decode offsets") mbp.Must(fin.Close(), "failed to close offsets") // If we're reading and writing to the same offsets path, move the input @@ -118,12 +116,6 @@ func (cmd *cmdJournalRead) Execute([]string) error { mbp.Must(os.Rename(cmd.OffsetsPath, cmd.OffsetsPath+".previous"), "failed to rename previous offsets") } - // Copy to carry-through any offsets of --offsets which are not matched - // to a journal on this invocation. They might match again in the future, - // and it would be surprising to the user were we to forget them. - for j, o := range cmd.beginOffsets { - cmd.endOffsets[j] = o - } } var err error @@ -177,7 +169,7 @@ func (cmd *cmdJournalRead) Execute([]string) error { // Use a temporary to atomically create the offsets. var fout, err = os.Create(cmd.OffsetsOutPath + ".temp") mbp.Must(err, "failed to open offsets for writing") - mbp.Must(json.NewEncoder(fout).Encode(cmd.endOffsets), "failed to encode offsets") + mbp.Must(json.NewEncoder(fout).Encode(cmd.offsets), "failed to encode offsets") mbp.Must(fout.Sync(), "failed to sync offset file") mbp.Must(fout.Close(), "failed to close offset file") mbp.Must(os.Rename(cmd.OffsetsOutPath+".temp", cmd.OffsetsOutPath), @@ -195,6 +187,9 @@ func (cmd *cmdJournalRead) listRefreshed(ctx context.Context, rjc pb.RoutedJourn nextFns = make(map[pb.Journal]context.CancelFunc) ) for _, j := range list.List().Journals { + if j.Spec.Suspend.GetLevel() == pb.JournalSpec_Suspend_FULL { + continue // Filter suspended journals (they have no content). + } if fn, ok := prevFns[j.Spec.Name]; ok { // Reader has already been started for this journal. nextFns[j.Spec.Name] = fn @@ -202,9 +197,9 @@ func (cmd *cmdJournalRead) listRefreshed(ctx context.Context, rjc pb.RoutedJourn continue } - // If the journal's in |beginOffsets|, use that as our initial offset. + // If the journal's in |offsets|, use that as our initial offset. // Otherwise if |Tail|, use -1. Else use zero. - var offset, ok = cmd.beginOffsets[j.Spec.Name] + var offset, ok = cmd.offsets[j.Spec.Name] if !ok && cmd.Tail { offset = -1 } @@ -245,7 +240,7 @@ func (cmd *cmdJournalRead) pumpFinished(rr *client.RetryReader, err error, nextC "offset": rr.Offset(), }).Info("read finished") - cmd.endOffsets[rr.Journal()] = rr.Offset() + cmd.offsets[rr.Journal()] = rr.Offset() delete(cmd.cancelFns, rr.Journal()) close(nextCh) diff --git a/cmd/gazctl/gazctlcmd/journals_suspend.go b/cmd/gazctl/gazctlcmd/journals_suspend.go new file mode 100644 index 00000000..fe3fb871 --- /dev/null +++ b/cmd/gazctl/gazctlcmd/journals_suspend.go @@ -0,0 +1,92 @@ +package gazctlcmd + +import ( + "context" + "fmt" + + log "github.com/sirupsen/logrus" + "go.gazette.dev/core/broker/client" + pb "go.gazette.dev/core/broker/protocol" + mbp "go.gazette.dev/core/mainboilerplate" + "golang.org/x/sync/errgroup" +) + +type cmdJournalSuspend struct { + Selector string `long:"selector" short:"l" required:"true" description:"Label Selector query to filter on"` + Force bool `long:"force" short:"f" description:"Suspend journals even if they have local fragments"` +} + +func init() { + CommandRegistry.AddCommand("journals", "suspend", "Suspend idle journals", ` +Suspend idle journals to use fewer or zero broker replicas. + +Suspension updates the 'suspend' field of the JournalSpec with its suspended +status and its resumption offset. When applying other updates to JournalSpecs, +operators utilizing journal suspension must take care to not overwrite the +journal's suspension configuration. + +Typically this means reading the current JournalSpec and its ModRevision, +copying the current 'suspend' field alongside other changes being made, +and then applying the updated JournalSpec with ExpectModRevision. + +The 'journals edit' subcommand uses this workflow and is safe to use with suspension. +`, &cmdJournalSuspend{}) +} + +func (cmd *cmdJournalSuspend) Execute([]string) error { + startup(JournalsCfg.BaseConfig) + + var err error + var ctx = context.Background() + var rjc = JournalsCfg.Broker.MustRoutedJournalClient(ctx) + + // Get the list of journals which match this selector. + var listRequest pb.ListRequest + listRequest.Selector, err = pb.ParseLabelSelector(cmd.Selector) + mbp.Must(err, "failed to parse label selector", "selector", cmd.Selector) + + var listResp *pb.ListResponse + listResp, err = client.ListAllJournals(ctx, rjc, listRequest) + mbp.Must(err, "failed to resolve journals from selector", cmd.Selector) + + group, ctx := errgroup.WithContext(ctx) + + for _, journal := range listResp.Journals { + if journal.Spec.Suspend.GetLevel() == pb.JournalSpec_Suspend_FULL { + continue + } + group.Go(func() error { + return suspendJournal(ctx, rjc, journal.Spec.Name, cmd.Force) + }) + } + return group.Wait() +} + +func suspendJournal( + ctx context.Context, + rjc pb.RoutedJournalClient, + journal pb.Journal, + force bool, +) error { + + var mode = pb.AppendRequest_SUSPEND_IF_FLUSHED + if force { + mode = pb.AppendRequest_SUSPEND_NOW + } + + var a = client.NewAppender(ctx, rjc, pb.AppendRequest{ + Journal: journal, + Suspend: mode, + }) + var err = a.Close() + + if err == nil || a.Response.Status == pb.Status_SUSPENDED { + log.WithFields(log.Fields{ + "journal": journal, + "status": a.Response.Status, + }).Info("journal suspension status") + return nil + } else { + return fmt.Errorf("failed to suspend journal %s: %w", journal, err) + } +} diff --git a/cmd/gazette/main.go b/cmd/gazette/main.go index 96844ac1..fcf13c48 100644 --- a/cmd/gazette/main.go +++ b/cmd/gazette/main.go @@ -38,6 +38,7 @@ var Config = new(struct { DisableStores bool `long:"disable-stores" env:"DISABLE_STORES" description:"Disable use of any configured journal fragment stores. The broker will neither list or persist remote fragments, and all data is discarded on broker exit."` WatchDelay time.Duration `long:"watch-delay" env:"WATCH_DELAY" default:"30ms" description:"Delay applied to the application of watched Etcd events. Larger values amortize the processing of fast-changing Etcd keys."` AuthKeys string `long:"auth-keys" env:"AUTH_KEYS" description:"Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all keys) Authorization tokens." json:"-"` + AutoSuspend bool `long:"auto-suspend" env:"AUTO_SUSPEND" description:"Automatically suspend journals which have persisted all fragments"` } `group:"Broker" namespace:"broker" env-namespace:"BROKER"` Etcd struct { @@ -105,10 +106,11 @@ func (cmdServe) Execute(args []string) error { fragment.FileSystemStoreRoot = Config.Broker.FileRoot } - broker.MinAppendRate = int64(Config.Broker.MinAppendRate) + broker.AutoSuspend = Config.Broker.AutoSuspend broker.MaxAppendRate = int64(Config.Broker.MaxAppendRate) - pb.MaxReplication = int32(Config.Broker.MaxReplication) + broker.MinAppendRate = int64(Config.Broker.MinAppendRate) fragment.DisableStores = Config.Broker.DisableStores + pb.MaxReplication = int32(Config.Broker.MaxReplication) var ( lo = pb.NewAuthJournalClient(pb.NewJournalClient(srv.GRPCLoopback), authorizer) From 950d95a158921af1462d52598943f192c4a3845f Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 9 Jan 2025 13:28:14 -0600 Subject: [PATCH 5/9] broker: fix a proxyRead shutdown race which could hang a goroutine Previously, a `stopCh` signal could hang the "pump" goroutine, sending into a channel which is never read. Instead, ensure we fully consume the pump channel and use a context cancellation to ensure it exits on `stopCh`. --- broker/read_api.go | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/broker/read_api.go b/broker/read_api.go index 39ff8cc2..d1a38f9c 100644 --- a/broker/read_api.go +++ b/broker/read_api.go @@ -1,6 +1,7 @@ package broker import ( + "context" "io" "net" @@ -56,7 +57,10 @@ func (svc *Service) Read(claims pb.Claims, req *pb.ReadRequest, stream pb.Journa func proxyRead(stream grpc.ServerStream, req *pb.ReadRequest, jc pb.JournalClient, stopCh <-chan struct{}) error { // We verified the client's authorization and are running under its context. // pb.AuthJournalClient will self-sign claims to proxy this journal on the client's behalf. - var ctx = pb.WithDispatchRoute(stream.Context(), req.Header.Route, req.Header.ProcessId) + var ctx, cancel = context.WithCancel(stream.Context()) + ctx = pb.WithDispatchRoute(ctx, req.Header.Route, req.Header.ProcessId) + + defer cancel() // We use the |stream| context for this RPC, which means a cancellation from // our client automatically propagates to the proxy |client| stream. @@ -68,42 +72,38 @@ func proxyRead(stream grpc.ServerStream, req *pb.ReadRequest, jc pb.JournalClien // Start a "pump" of |client| reads that we'll select from. go func() { - var resp pb.ReadResponse for { - var err = client.RecvMsg(&resp) - - select { - case chunkCh <- proxyChunk{resp: resp, err: err}: - if err != nil { - return - } - case <-ctx.Done(): - return // RPC complete. + var resp, err = client.Recv() + chunkCh <- proxyChunk{resp: resp, err: err} + + if err != nil { + return } } }() - // Read and proxy chunks from |client|, or immediately halt with EOF - // if |stopCh| is signaled. + // Read and proxy chunks from `client`. + // Cancel the proxy RPC when `stopCh` is signaled. var chunk proxyChunk for { select { case chunk = <-chunkCh: - if chunk.err == io.EOF { + if chunk.err == nil { + _ = stream.SendMsg(chunk.resp) + } else if chunk.err == io.EOF || ctx.Err() != nil { return nil } else if chunk.err != nil { return chunk.err - } else if err = stream.SendMsg(&chunk.resp); err != nil { - return err } case <-stopCh: - return nil + stopCh = nil + cancel() } } } type proxyChunk struct { - resp pb.ReadResponse + resp *pb.ReadResponse err error } From 501b9fc3d813fe21b1359b0534201f97ddf776da Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 9 Jan 2025 13:41:27 -0600 Subject: [PATCH 6/9] significant updates to CI and build infrastructure Junk the docker-based builder strategy and instead use the GitHub Actions CI environment for builds. Introduce an ARM64 broker variant using cross-compilation and the Github docker build action with QEMU. Switch to using the librocksdb8.9 library provided by Ubuntu 24.04 within CI and within the "examples" image, rather than pinning RocksDB and building it ourselves. Switch to Github container registry for release builds. --- .circleci/config.yml | 70 ----------- .github/workflows/ci-workflow.yaml | 156 +++++------------------- .vscode/settings.json | 3 + Dockerfile | 53 ++++++++ Makefile | 189 +++++++++++++++++++++-------- mk/ci-builder.Dockerfile | 69 ----------- mk/ci-release.Dockerfile | 25 ---- mk/cmd-reference.mk | 60 --------- mk/common-build.mk | 136 --------------------- mk/common-config.mk | 41 ------- mk/microk8s.mk | 13 -- 11 files changed, 224 insertions(+), 591 deletions(-) delete mode 100644 .circleci/config.yml create mode 100644 .vscode/settings.json create mode 100644 Dockerfile delete mode 100644 mk/ci-builder.Dockerfile delete mode 100644 mk/ci-release.Dockerfile delete mode 100644 mk/cmd-reference.mk delete mode 100644 mk/common-build.mk delete mode 100644 mk/common-config.mk delete mode 100644 mk/microk8s.mk diff --git a/.circleci/config.yml b/.circleci/config.yml deleted file mode 100644 index f80db6ca..00000000 --- a/.circleci/config.yml +++ /dev/null @@ -1,70 +0,0 @@ -version: 2 -jobs: - build: - docker: - - image: gazette/ci-builder:20200421 - environment: - # Instruct the go tool to place the module cache under the working - # directory (~/project) - GOPATH: /root/project/.build/go-path - # RocksDB version to build. Keep in sync with mk/config.mk - # Also be sure to update the save_cache step below. - ROCKSDB_VERSION: 6.22.1 - # `nproc` on CircleCI docker runners returns the number of host cores, - # not provisioned cores (eg, 36 cores when only 2 are available). - NPROC: 2 - # Same issue with GOMAXPROCS. - GOMAXPROCS: 2 - steps: - - checkout - - setup_remote_docker - - restore_cache: - # We can't use a variable here :( Must be hard-coded. - # https://discuss.circleci.com/t/cannot-use-circle-yml-environment-variables-in-cache-keys/10994 - key: rocksdb-v1-v6.22.1 - - run: - name: Build RocksDB - command: make $(pwd)/.build/rocksdb-v${ROCKSDB_VERSION}/librocksdb.so - - save_cache: - name: Cache RocksDB - key: rocksdb-v1-v6.22.1 - paths: - # We *also* can't use a variable here (seriously?!). - # https://discuss.circleci.com/t/can-i-use-variables-in-cache-paths/11393 - - .build/rocksdb-v6.22.1/ - - restore_cache: - keys: - - go-mod-v1-{{ checksum "go.sum" }} - - go-mod-v1- - - run: - name: Go Install - command: make go-install - - run: - name: Go Test - command: make go-test-ci - - save_cache: - name: Cache Go Modules - key: go-mod-v1-{{ checksum "go.sum" }} - paths: - - .build/go-path/pkg - release: - docker: - - image: gazette/ci-builder:20200421 - steps: - - run: - name: Publish Gazette - command: echo "release tag ${CIRCLE_TAG}" - -workflows: - version: 2 - build: - jobs: - - build - release: - jobs: - - release: - filters: - tags: - only: /^v.*/ - branches: - ignore: /.*/ diff --git a/.github/workflows/ci-workflow.yaml b/.github/workflows/ci-workflow.yaml index cdfd870d..535c0d72 100644 --- a/.github/workflows/ci-workflow.yaml +++ b/.github/workflows/ci-workflow.yaml @@ -1,139 +1,47 @@ name: Gazette Continuous Integration -# We build on any push to a branch, or when a release is created. on: - pull_request: - paths-ignore: - - "docs/**" push: - branches: - - "master" - # Ignore pushes to tags, since those ought to be handled by the release created event. - tags-ignore: - - "*" - paths-ignore: - - "docs/**" - release: - # Without this additional restriction, GH actions will trigger multiple runs for a single - # release, because it fires off separate events creating vs publishing the release. - types: [created] - -env: - # This is only used as the cache key to prevent rebuilding rocksdb every time. Eventually - # we'll need to figure out a solution that doesn't duplicate this version everywhere. - # For now, ensure that it's changed both here and in mk/common-config.mk. - ROCKSDB_VERSION: "6.22.1" + branches: [master] + pull_request: + branches: [master] jobs: build: - name: "Build" - runs-on: ubuntu-22.04 + runs-on: ubuntu-24.04 steps: - - name: "Checkout" - uses: actions/checkout@v2 - - # Sets outputs for version, docker tag, and whether we should push images and upload release - # artifacts. These outputs are used by later steps. - - name: "Release Info" - id: release_info - env: - # just having this in the env is enough to make it visible in the "raw" logs attached to the run - GITHUB_EVENT_JSON: "${{ toJson(github.event) }}" - run: | - is_release=${{ github.event_name == 'release' }} - if [[ "$is_release" == "true" ]]; then - push_images=true - tag_name=${{ github.event.release.tag_name }} - # The version regex doesn't need to be super sophisticated, since the goal is not to - # validate that this is a semantic version number. Rather the goal is just to see if - # matches the typical pattern we use for release tags (e.g. v0.86.1). If it does, then - # we'll remove the 'v' prefix and use the remainder as the docker tag - # If a release tag does not match that format, then we'll just use the tag value as is. - if echo "$tag_name" | grep -qE '^v[0-9]+\.[0-9]+\.[0-9]+'; then - version="${tag_name#v}" - else - version="${tag_name}" - fi - else - # This is not a release, so we'll use 'dev-' for the version number - # and just 'latest-dev' for the docker tag. - sha=${{ github.sha }} - version="dev-${sha:0:7}" - # If this is a master build, then we'll treat this as a release and just use the - # hard-coded tag as the docker image tag. - if [[ '${{ github.ref }}' == 'refs/heads/master' ]]; then - # We don't want to put the git sha in the docker tag because otherwise they'll - # accumulate forever and just clutter up the page on docker hub. So 'latest-dev' - # just always gets you the most recent master build, and if you want a specific master - # build, then you can use the '@sha256:...' syntax. - docker_tag="latest-dev" - push_images='true' - else - push_images='false' - fi - fi - echo ::set-output name=VERSION::${version} - echo ::set-output name=DOCKER_TAG::${docker_tag:-$version} - echo ::set-output name=PUSH_IMAGES::${push_images} - echo ::set-output name=IS_RELEASE::${is_release} - - # Try to load the ci-builder docker image from the cache. If this misses, then it will be - # built automatically by a make rule. A later step will then run 'docker save' to export the - # image as a tar archive so we can cache it. - - name: "CI Builder Docker Image Cache" - id: "ci_builder_cache" - uses: actions/cache@v2 + - uses: actions/checkout@v4 with: - key: ci-builder-${{ hashFiles('mk/ci-builder.Dockerfile') }} - path: ".build/ci-builder-image.tar" + fetch-depth: 0 # Full history. - # The 'c' in these Cache steps is just for changing the cache key so that - # we can manually invalidate the cache if we need. - - name: "RocksDB Cache" - uses: actions/cache@v2 - with: - key: rocksdb-c4-${{ env.ROCKSDB_VERSION }} - path: ".build-ci/rocksdb-v${{ env.ROCKSDB_VERSION }}" - - - name: "Go Module Cache" - uses: actions/cache@v2 - with: - key: go-mod-c4-${{ hashFiles('go.sum') }} - path: ".build-ci/go-path/pkg" - # If we don't have a cached directory that matches the hash exactly, - # then this will allow a non-matching directory to be pulled in. This is safe - # because go will use its own finer-grained cache invalidation logic. - restore-keys: "go-mod-c4-" + - uses: docker/setup-qemu-action@v3 + - uses: docker/setup-buildx-action@v3 + - name: Login to GitHub container registry + run: | + echo "${{ secrets.GITHUB_TOKEN }}" | \ + docker login --username ${{ github.actor }} --password-stdin ghcr.io - - name: "Build Binaries" - run: "make as-ci target=release-linux-binaries VERSION=${{ steps.release_info.outputs.VERSION }}" + - uses: actions/setup-go@v4 + - run: sudo apt install -y protobuf-compiler librocksdb-dev libsqlite3-dev etcd-server + - run: echo "VERSION=$(git describe --dirty --tags)" >> "$GITHUB_ENV" - - name: "Test" - run: "make as-ci target=go-test-ci VERSION=${{ steps.release_info.outputs.VERSION }}" + - run: make go-test-ci + - run: make go-build go-build-arm64 - # We upload this to the artifacts that are attached to the action just to make it easy for - # someone to pull down a build from another branch. - - name: "Upload Binaries" - uses: actions/upload-artifact@v4 + - uses: docker/build-push-action@v5 with: - name: "gazette-x86_64-linux-gnu.zip" - path: ".build-ci/gazette-x86_64-linux-gnu.zip" - - - name: "Upload Release Binaries" - if: steps.release_info.outputs.IS_RELEASE == 'true' - uses: actions/upload-release-asset@v1 - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + context: .build + file: Dockerfile + target: broker + platforms: linux/amd64,linux/arm64 + tags: "ghcr.io/gazette/broker:${{ env.VERSION }}" + push: true + + - uses: docker/build-push-action@v5 with: - asset_name: gazette-x86_64-linux-gnu.zip - asset_path: ".build-ci/gazette-x86_64-linux-gnu.zip" - upload_url: "${{ github.event.release.upload_url }}" - asset_content_type: application/zip - - - name: "Build and Push Docker Images" - if: steps.release_info.outputs.PUSH_IMAGES == 'true' - run: | - docker login -u '${{ secrets.DOCKER_USERNAME }}' -p '${{ secrets.DOCKER_PASSWORD }}' ${{ secrets.DOCKER_REGISTRY }} - make as-ci target=ci-release-gazette-examples VERSION=${{ steps.release_info.outputs.VERSION }} - make as-ci target=ci-release-gazette-broker VERSION=${{ steps.release_info.outputs.VERSION }} - make push-to-registry REGISTRY=${{ secrets.DOCKER_REGISTRY }} RELEASE_TAG=${{ steps.release_info.outputs.DOCKER_TAG }} + context: .build + file: Dockerfile + target: examples + platforms: linux/amd64 + tags: "ghcr.io/gazette/examples:${{ env.VERSION }}" + push: true \ No newline at end of file diff --git a/.vscode/settings.json b/.vscode/settings.json new file mode 100644 index 00000000..edeb32b8 --- /dev/null +++ b/.vscode/settings.json @@ -0,0 +1,3 @@ +{ + "go.buildTags": "libsqlite3" +} \ No newline at end of file diff --git a/Dockerfile b/Dockerfile new file mode 100644 index 00000000..00aadf08 --- /dev/null +++ b/Dockerfile @@ -0,0 +1,53 @@ +################################################################################ +# Gazette broker image. + +FROM ubuntu:24.04 AS broker + +ARG TARGETARCH + +RUN apt-get update -y \ + && apt-get upgrade -y \ + && apt-get install --no-install-recommends -y \ + ca-certificates \ + curl \ + && rm -rf /var/lib/apt/lists/* + +COPY ${TARGETARCH}/gazette ${TARGETARCH}/gazctl /usr/local/bin + +# Run as non-privileged "gazette" user. +RUN useradd gazette --create-home --shell /usr/sbin/nologin +USER gazette +WORKDIR /home/gazette + +################################################################################ +# Gazette examples image. + +FROM ubuntu:24.04 AS examples + +ARG TARGETARCH + +RUN apt-get update -y \ + && apt-get upgrade -y \ + && apt-get install --no-install-recommends -y \ + ca-certificates \ + curl \ + librocksdb8.9 \ + libsqlite3-0 \ + && rm -rf /var/lib/apt/lists/* + +# Copy binaries to the image. +COPY \ + ${TARGETARCH}/bike-share \ + ${TARGETARCH}/chunker \ + ${TARGETARCH}/counter \ + ${TARGETARCH}/gazctl \ + ${TARGETARCH}/integration.test \ + ${TARGETARCH}/summer \ + ${TARGETARCH}/wordcountctl \ + /usr/local/bin/ + +# Run as non-privileged "gazette" user. +RUN useradd gazette --create-home --shell /usr/sbin/nologin +USER gazette +WORKDIR /home/gazette + diff --git a/Makefile b/Makefile index 200a493d..a54dfbc7 100644 --- a/Makefile +++ b/Makefile @@ -1,23 +1,47 @@ -include mk/common-config.mk - -# Project-specific configuration: - -# Go binaries to package under the `gazette/broker` image. -ci-release-gazette-broker-targets = \ - ${WORKDIR}/go-path/bin/gazctl \ - ${WORKDIR}/go-path/bin/gazette - -# Go binaries to package under the `gazette/examples` image. -ci-release-gazette-examples-targets = \ - ${WORKDIR}/go-path/bin/bike-share \ - ${WORKDIR}/go-path/bin/chunker \ - ${WORKDIR}/go-path/bin/counter \ - ${WORKDIR}/go-path/bin/gazctl \ - ${WORKDIR}/go-path/bin/integration.test \ - ${WORKDIR}/go-path/bin/summer \ - ${WORKDIR}/go-path/bin/wordcountctl - -# Targets of protobufs which must be compiled. +# Git version & date which are injected into built binaries. +VERSION ?= "$(shell git describe --dirty --tags)" +DATE = $(shell date +%F-%T-%Z) +# Repository root (the directory of this Makefile). +ROOTDIR = $(abspath $(dir $(firstword $(MAKEFILE_LIST)))) +# Location to place intermediate files and output artifacts. +# Note the go tool ignores directories with leading '.' or '_'. +WORKDIR = ${ROOTDIR}/.build + +############################################################################### +# Rules for generating protobuf sources: + +${WORKDIR}/protoc-gen-gogo: + go mod download github.com/golang/protobuf + go build -o $@ github.com/gogo/protobuf/protoc-gen-gogo + +${WORKDIR}/protoc-gen-grpc-gateway: + go mod download github.com/grpc-ecosystem/grpc-gateway + go build -o $@ github.com/grpc-ecosystem/grpc-gateway/protoc-gen-grpc-gateway + +# PROTOC_INC_MODULES is an append-able list of Go modules +# which should be included with `protoc` invocations. +PROTOC_INC_MODULES += "github.com/golang/protobuf" +PROTOC_INC_MODULES += "github.com/gogo/protobuf" + +# module_path expands a $(module), like go.gazette.dev/core, to the local path +# of its repository as currently specified by go.mod. +module_path = $(shell go list -f '{{ .Dir }}' -m $(module)) + +# Run the protobuf compiler to generate message and gRPC service implementations. +# Invoke protoc with local and third-party include paths set. The `go list` tool +# is used to map submodules to corresponding go.mod versions and paths. +%.pb.go: %.proto ${WORKDIR}/protoc-gen-gogo + PATH=${WORKDIR}:$${PATH} ;\ + protoc -I . $(foreach module, $(PROTOC_INC_MODULES), -I$(module_path)) \ + --gogo_out=paths=source_relative,plugins=grpc:. \ + $*.proto + +%.pb.gw.go: %.proto ${WORKDIR}/protoc-gen-grpc-gateway + PATH=${WORKDIR}:$${PATH} ;\ + protoc -I . $(foreach module, $(PROTOC_INC_MODULES), -I$(module_path)) \ + --grpc-gateway_out=logtostderr=true,paths=source_relative,generate_unbound_methods=false,grpc_api_configuration=$*_gateway.yaml:. \ + $*.proto + protobuf-targets = \ ./broker/protocol/protocol.pb.go \ ./broker/protocol/protocol.pb.gw.go \ @@ -29,38 +53,97 @@ protobuf-targets = \ # consumer.proto depends on protocol.proto & recorded_op.proto. consumer/protocol/consumer.pb.go: broker/protocol/protocol.proto consumer/recoverylog/recorded_op.proto -# Rule for integration.test, depended on by ci-release-gazette-examples. -# It's an actual file, but we set it as PHONY so it's kept fresh -# with each build of that target. -${WORKDIR}/go-path/bin/integration.test: +############################################################################### +# Rules for building and testing: + +${WORKDIR}/amd64/integration.test: go test -v -c -tags integration ./test/integration -o $@ -include mk/common-build.mk -include mk/microk8s.mk -include mk/cmd-reference.mk - -# Push the broker & example image to a specified private registry. -# Override the registry to use by passing a "REGISTRY=" flag to make. -REGISTRY=localhost:32000 -RELEASE_TAG=latest -push-to-registry: - docker tag gazette/broker:latest $(REGISTRY)/broker:$(RELEASE_TAG) - docker tag gazette/examples:latest $(REGISTRY)/examples:$(RELEASE_TAG) - docker push $(REGISTRY)/broker:$(RELEASE_TAG) - docker push $(REGISTRY)/examples:$(RELEASE_TAG) - -${WORKDIR}/gazette-x86_64-linux-gnu.zip: go-install - cd ${WORKDIR}/go-path/bin/ - zip gazette-x86_64-linux-gnu.zip gazette gazctl - -# Builds a zip file containing both the gazette and gazctl release binaries. This target may only be run -# on a linux host, since we don't currently support cross-compilation (we may want to in the -# future). -release-linux-binaries: go-install - @# sanity check, since our make builds don't support cross-compilation at the moment - @test "$(shell uname -io)" = "x86_64 GNU/Linux" || (echo "only x86_64 linux binaries are produced" && exit 1) - @rm -f ${WORKDIR}/gazette-x86_64-linux-gnu.zip - zip -j ${WORKDIR}/gazette-x86_64-linux-gnu.zip ${WORKDIR}/go-path/bin/gazette ${WORKDIR}/go-path/bin/gazctl - - -.PHONY: release-linux-binaries ${WORKDIR}/go-path/bin/integration.test +go-build: $(protobuf-targets) ${WORKDIR}/amd64/integration.test + MBP=go.gazette.dev/core/mainboilerplate ;\ + go build -o ${WORKDIR}/amd64/ -v --tags libsqlite3 \ + -ldflags "-X $${MBP}.Version=${VERSION} -X $${MBP}.BuildDate=${DATE}" ./... + +go-build-arm64: $(protobuf-targets) + MBP=go.gazette.dev/core/mainboilerplate ;\ + GOOS=linux ;\ + GOARCH=arm64 \ + go build -o ${WORKDIR}/arm64/ -v --tags nozstd ./cmd/gazette ./cmd/gazctl + +go-test-ci: ${protobuf-targets} + GORACE="halt_on_error=1" go test -race -count=10 --tags libsqlite3 --failfast ./... + +go-test-fast: ${protobuf-targets} + go test --tags libsqlite3 ./... + +############################################################################### +# Rules for updating reference documentation: + +cmd-reference-targets = \ + docs/_static/cmd-gazette-serve.txt \ + docs/_static/cmd-gazette-print-config.txt \ + docs/_static/cmd-gazctl.txt \ + docs/_static/cmd-gazctl-attach-uuids.txt \ + docs/_static/cmd-gazctl-journals-append.txt \ + docs/_static/cmd-gazctl-journals-apply.txt \ + docs/_static/cmd-gazctl-journals-edit.txt \ + docs/_static/cmd-gazctl-journals-fragments.txt \ + docs/_static/cmd-gazctl-journals-list.txt \ + docs/_static/cmd-gazctl-journals-prune.txt \ + docs/_static/cmd-gazctl-journals-read.txt \ + docs/_static/cmd-gazctl-journals-reset-head.txt \ + docs/_static/cmd-gazctl-journals-suspend.txt \ + docs/_static/cmd-gazctl-print-config.txt \ + docs/_static/cmd-gazctl-shards-apply.txt \ + docs/_static/cmd-gazctl-shards-edit.txt \ + docs/_static/cmd-gazctl-shards-list.txt \ + docs/_static/cmd-gazctl-shards-prune.txt + +cmd-reference: ${cmd-reference-targets} + + +docs/_static/cmd-gazette-serve.txt: go-build + ${WORKDIR}/amd64/gazette serve --help > $@ || true +docs/_static/cmd-gazette-print-config.txt: go-build + ${WORKDIR}/amd64/gazette serve print config --help > $@ || true + +docs/_static/cmd-gazctl.txt: go-build + ${WORKDIR}/amd64/gazctl --help > $@ || true +docs/_static/cmd-gazctl-attach-uuids.txt: go-build + ${WORKDIR}/amd64/gazctl attach-uuids --help > $@ || true +docs/_static/cmd-gazctl-journals-append.txt: go-build + ${WORKDIR}/amd64/gazctl journals append --help > $@ || true +docs/_static/cmd-gazctl-journals-apply.txt: go-build + ${WORKDIR}/amd64/gazctl journals apply --help > $@ || true +docs/_static/cmd-gazctl-journals-edit.txt: go-build + ${WORKDIR}/amd64/gazctl journals edit --help > $@ || true +docs/_static/cmd-gazctl-journals-fragments.txt: go-build + ${WORKDIR}/amd64/gazctl journals fragments --help > $@ || true +docs/_static/cmd-gazctl-journals-list.txt: go-build + ${WORKDIR}/amd64/gazctl journals list --help > $@ || true +docs/_static/cmd-gazctl-journals-prune.txt: go-build + ${WORKDIR}/amd64/gazctl journals prune --help > $@ || true +docs/_static/cmd-gazctl-journals-read.txt: go-build + ${WORKDIR}/amd64/gazctl journals read --help > $@ || true +docs/_static/cmd-gazctl-journals-reset-head.txt: go-build + ${WORKDIR}/amd64/gazctl journals reset-head --help > $@ || true +docs/_static/cmd-gazctl-journals-suspend.txt: go-build + ${WORKDIR}/amd64/gazctl journals suspend --help > $@ || true +docs/_static/cmd-gazctl-print-config.txt: go-build + ${WORKDIR}/amd64/gazctl print-config --help > $@ || true +docs/_static/cmd-gazctl-shards-apply.txt: go-build + ${WORKDIR}/amd64/gazctl shards apply --help > $@ || true +docs/_static/cmd-gazctl-shards-edit.txt: go-build + ${WORKDIR}/amd64/gazctl shards edit --help > $@ || true +docs/_static/cmd-gazctl-shards-list.txt: go-build + ${WORKDIR}/amd64/gazctl shards list --help > $@ || true +docs/_static/cmd-gazctl-shards-prune.txt: go-build + ${WORKDIR}/amd64/gazctl shards prune --help > $@ || true + + +.PHONY: \ + cmd-reference \ + go-build \ + go-build-arm64 \ + go-test-ci \ + go-test-fast \ No newline at end of file diff --git a/mk/ci-builder.Dockerfile b/mk/ci-builder.Dockerfile deleted file mode 100644 index e341a266..00000000 --- a/mk/ci-builder.Dockerfile +++ /dev/null @@ -1,69 +0,0 @@ -# Note that this image will be cached in Github Actions, and the cache key is computed by hashing -# this file. This works only so long as there are no _other_ files that go into the final image. -# So if you add any ADD or COPY directives, be sure to update the cache key in the github actions -# workflow yaml -FROM ubuntu:22.04 - -RUN apt-get update -y \ - && apt-get upgrade -y \ - && apt-get install --no-install-recommends -y \ - build-essential \ - ca-certificates \ - curl \ - git \ - libbz2-dev \ - libjemalloc-dev \ - liblz4-dev \ - libprotobuf-dev \ - libsnappy-dev \ - libsqlite3-dev \ - libzstd-dev \ - protobuf-compiler \ - zlib1g-dev \ - zip \ - && rm -rf /var/lib/apt/lists/* - -ARG GOLANG_VERSION=1.22.4 -ARG GOLANG_SHA256=ba79d4526102575196273416239cca418a651e049c2b099f3159db85e7bade7d - -ARG DOCKER_VERSION=19.03.8 -ARG DOCKER_SHA256=7f4115dc6a3c19c917f8b9664d7b51c904def1c984e082c4600097433323cf6f - -ARG ETCD_VERSION=v3.5.0 -ARG ETCD_SHA256=864baa0437f8368e0713d44b83afe21dce1fb4ee7dae4ca0f9dd5f0df22d01c4 - -ENV PATH=/usr/local/go/bin:$PATH - -RUN curl -L -o /tmp/golang.tgz \ - https://golang.org/dl/go${GOLANG_VERSION}.linux-amd64.tar.gz \ - && echo "${GOLANG_SHA256} /tmp/golang.tgz" | sha256sum -c - \ - && tar --extract \ - --file /tmp/golang.tgz \ - --directory /usr/local \ - && rm /tmp/golang.tgz \ - && go version - -RUN curl -L -o /tmp/docker.tgz \ - https://download.docker.com/linux/static/stable/x86_64/docker-${DOCKER_VERSION}.tgz \ - && echo "${DOCKER_SHA256} /tmp/docker.tgz" | sha256sum -c - \ - && tar --extract \ - --file /tmp/docker.tgz \ - --strip-components 1 \ - --directory /usr/local/bin/ \ - && rm /tmp/docker.tgz \ - && docker --version - -RUN curl -L -o /tmp/etcd.tgz \ - https://github.com/etcd-io/etcd/releases/download/${ETCD_VERSION}/etcd-${ETCD_VERSION}-linux-amd64.tar.gz \ - && echo "${ETCD_SHA256} /tmp/etcd.tgz" | sha256sum -c - \ - && tar --extract \ - --file /tmp/etcd.tgz \ - --directory /tmp/ \ - && mv /tmp/etcd-${ETCD_VERSION}-linux-amd64/etcd /tmp/etcd-${ETCD_VERSION}-linux-amd64/etcdctl /usr/local/bin \ - && chown 1000:1000 /usr/local/bin/etcd /usr/local/bin/etcdctl \ - && rm -r /tmp/etcd-${ETCD_VERSION}-linux-amd64/ \ - && rm /tmp/etcd.tgz \ - && etcd --version - -WORKDIR /gazette - diff --git a/mk/ci-release.Dockerfile b/mk/ci-release.Dockerfile deleted file mode 100644 index 74bbab62..00000000 --- a/mk/ci-release.Dockerfile +++ /dev/null @@ -1,25 +0,0 @@ -FROM ubuntu:22.04 - -# Pick run-time library packages which match the development packages -# used by the ci-builder image. "curl" is included, to allow node-zone.sh -# mappings to directly query AWS/Azure/GCP metadata APIs. -RUN apt-get update -y \ - && apt-get upgrade -y \ - && apt-get install --no-install-recommends -y \ - ca-certificates \ - curl \ - libgflags2.2 \ - libjemalloc2 \ - libsnappy1v5 \ - libzstd1 \ - && rm -rf /var/lib/apt/lists/* - -# Copy binaries & librocks.so to the image. Configure Rocks for run-time linking. -COPY * /usr/local/bin/ -RUN mv /usr/local/bin/librocksdb.so* /usr/local/lib/ && ldconfig - -# Run as non-privileged "gazette" user. -RUN useradd gazette --create-home --shell /usr/sbin/nologin -USER gazette -WORKDIR /home/gazette - diff --git a/mk/cmd-reference.mk b/mk/cmd-reference.mk deleted file mode 100644 index 7e96b887..00000000 --- a/mk/cmd-reference.mk +++ /dev/null @@ -1,60 +0,0 @@ - -cmd-reference-targets = \ - docs/_static/cmd-gazette-serve.txt \ - docs/_static/cmd-gazette-print-config.txt \ - docs/_static/cmd-gazctl.txt \ - docs/_static/cmd-gazctl-attach-uuids.txt \ - docs/_static/cmd-gazctl-journals-append.txt \ - docs/_static/cmd-gazctl-journals-apply.txt \ - docs/_static/cmd-gazctl-journals-edit.txt \ - docs/_static/cmd-gazctl-journals-fragments.txt \ - docs/_static/cmd-gazctl-journals-list.txt \ - docs/_static/cmd-gazctl-journals-prune.txt \ - docs/_static/cmd-gazctl-journals-read.txt \ - docs/_static/cmd-gazctl-journals-reset-head.txt \ - docs/_static/cmd-gazctl-print-config.txt \ - docs/_static/cmd-gazctl-shards-apply.txt \ - docs/_static/cmd-gazctl-shards-edit.txt \ - docs/_static/cmd-gazctl-shards-list.txt \ - docs/_static/cmd-gazctl-shards-prune.txt - -cmd-reference: ${cmd-reference-targets} - - -docs/_static/cmd-gazette-serve.txt: go-install - gazette serve --help > $@ || true -docs/_static/cmd-gazette-print-config.txt: go-install - gazette serve print config --help > $@ || true - -docs/_static/cmd-gazctl.txt: go-install - gazctl --help > $@ || true -docs/_static/cmd-gazctl-attach-uuids.txt: go-install - gazctl attach-uuids --help > $@ || true -docs/_static/cmd-gazctl-journals-append.txt: go-install - gazctl journals append --help > $@ || true -docs/_static/cmd-gazctl-journals-apply.txt: go-install - gazctl journals apply --help > $@ || true -docs/_static/cmd-gazctl-journals-edit.txt: go-install - gazctl journals edit --help > $@ || true -docs/_static/cmd-gazctl-journals-fragments.txt: go-install - gazctl journals fragments --help > $@ || true -docs/_static/cmd-gazctl-journals-list.txt: go-install - gazctl journals list --help > $@ || true -docs/_static/cmd-gazctl-journals-prune.txt: go-install - gazctl journals prune --help > $@ || true -docs/_static/cmd-gazctl-journals-read.txt: go-install - gazctl journals read --help > $@ || true -docs/_static/cmd-gazctl-journals-reset-head.txt: go-install - gazctl journals reset-head --help > $@ || true -docs/_static/cmd-gazctl-print-config.txt: go-install - gazctl print-config --help > $@ || true -docs/_static/cmd-gazctl-shards-apply.txt: go-install - gazctl shards apply --help > $@ || true -docs/_static/cmd-gazctl-shards-edit.txt: go-install - gazctl shards edit --help > $@ || true -docs/_static/cmd-gazctl-shards-list.txt: go-install - gazctl shards list --help > $@ || true -docs/_static/cmd-gazctl-shards-prune.txt: go-install - gazctl shards prune --help > $@ || true - -.PHONY: cmd-reference diff --git a/mk/common-build.mk b/mk/common-build.mk deleted file mode 100644 index 4577fccf..00000000 --- a/mk/common-build.mk +++ /dev/null @@ -1,136 +0,0 @@ -# common-build.mk defines build rules which are common to both -# go.gazette.dev/core, and external consumer application projects -# which re-use the Gazette build infrastructure. - -# The ci-builder-image target builds a Docker image suitable for building -# gazette. It is the primary image used by gazette continuous integration builds. -ci-builder-image: ${WORKDIR}/ci-builder-image.tar - docker load -i ${WORKDIR}/ci-builder-image.tar - -# Builds the ci-builder docker image and also saves it as a tar file in the build directory -# This allows us to skip building the image without docker needing to check each individual -# layer, and also allows the tar file to be cached -${WORKDIR}/ci-builder-image.tar: - docker build -t gazette/ci-builder:latest - < ${COREDIR}/mk/ci-builder.Dockerfile - mkdir -p ${WORKDIR} - docker save -o ${WORKDIR}/ci-builder-image.tar gazette/ci-builder:latest - -host_os=$(shell uname -s) -AS_CI_RUN_ARGS= -ifeq ($(host_os),Linux) - AS_CI_RUN_ARGS = --user $(shell id -u):$(shell id -g) --group-add $(shell stat -c '%g' /var/run/docker.sock) -endif - -# The as-ci rule recursively calls `make` _within_ a instance of the ci-builder-image, -# and bind-mounting the gazette repository into the container. This rule allows for -# idempotent gazette builds which exactly match those produced by the CI builder. -# It uses a few tricks to keep builds fast: -# * The gazette repository checkout is bind-mounted into the container. -# * A ${WORKDIR}-ci directory is bind-mounted under the containers relative work -# directory. This keeps intermediate files of local vs "as-ci" builds separate -# so they don't clobber one another. -# * The GOPATH and GOCACHE variables are set to fall within the mounted work -# directory. Notably, this means repeat invocations can re-use the go modules -# cache, and the go build cache. -# * The Host's Docker socket is bind-mounted into the container, which has a docker -# client. This allows the ci-builder container to itself build Docker images. -# -# This will always run 'go mod download' before running the desired make target. This is -# done because users of gazette libraries are also depending on this makefile and so -# they need go to download the modules so that these files will be present before calling -# make. End-to-end, the process is to use 'go mod download' to download the makefile within -# the container, then the consumer's makefile will import this file, then make continues -# to build the target. -as-ci: ci-builder-image - mkdir -p ${WORKDIR} ${WORKDIR}-ci - # Strip root prefix from WORKDIR to build its equivalent within the container. - ROOT_CI=/gazette ;\ - WORK_CI=$${ROOT_CI}$(subst ${ROOTDIR},,${WORKDIR}) ;\ - docker run ${AS_CI_RUN_ARGS} \ - --rm \ - --tty \ - --mount src=${WORKDIR}-ci,target=$${WORK_CI},type=bind \ - --mount src=${ROOTDIR},target=$${ROOT_CI},type=bind \ - --env GOPATH=$${WORK_CI}/go-path \ - --env GOCACHE=$${WORK_CI}/go-build-cache \ - --mount src=/var/run/docker.sock,target=/var/run/docker.sock,type=bind \ - gazette/ci-builder /bin/sh -ec \ - "go mod download && \ - make ${target} VERSION=${VERSION} DATE=${DATE} REGISTRY=${REGISTRY} RELEASE_TAG=${RELEASE_TAG}" - -# Go build & test targets. -go-install: $(ROCKSDIR)/librocksdb.so $(protobuf-targets) - MBP=go.gazette.dev/core/mainboilerplate ;\ - go install -v --tags "${GO_BUILD_TAGS}" \ - -ldflags "-X $${MBP}.Version=${VERSION} -X $${MBP}.BuildDate=${DATE}" ./... -go-test-fast: ${ROCKSDIR}/librocksdb.so ${protobuf-targets} - go test -p ${NPROC} --tags "${GO_BUILD_TAGS}" ./... -go-test-ci: ${ROCKSDIR}/librocksdb.so ${protobuf-targets} - GORACE="halt_on_error=1" go test -p ${NPROC} -race -count=10 --tags "${GO_BUILD_TAGS}" --failfast ./... - -# The ci-release-% implicit rule builds a Docker image named by the rule -# stem, using binaries enumerated by a `-target` suffix. For example, -# an invocation with `ci-release-gazette-examples` has a stem `gazette-examples`, and will -# package binaries listed in `ci-release-gazette-examples-targets` into a docker -# image named `gazette/examples:latest`. -.SECONDEXPANSION: -ci-release-%: $(ROCKSDIR)/librocksdb.so go-install $$($$@-targets) - rm -rf ${WORKDIR}/ci-release - mkdir -p ${WORKDIR}/ci-release - ln ${$@-targets} ${ROCKSDIR}/librocksdb.so.${ROCKSDB_VERSION} \ - ${WORKDIR}/ci-release - docker build \ - -f ${COREDIR}/mk/ci-release.Dockerfile \ - -t $(subst -,/,$*):latest \ - ${WORKDIR}/ci-release/ - -# The librocksdb.so fetches and builds the version of RocksDB identified by -# the rule stem (eg, 5.17.2). We require a custom rule to build RocksDB as -# it's necessary to build with run-time type information (USE_RTTI=1), which -# is not enabled by default in third-party packages. -${WORKDIR}/rocksdb-v%/librocksdb.so: - # Fetch RocksDB source. - mkdir -p ${WORKDIR}/rocksdb-v$* - curl -L -o ${WORKDIR}/tmp.tgz https://github.com/facebook/rocksdb/archive/v$*.tar.gz - tar xzf ${WORKDIR}/tmp.tgz -C ${WORKDIR}/rocksdb-v$* --strip-components=1 - rm ${WORKDIR}/tmp.tgz - @# PORTABLE=1 prevents rocks from passing `-march=native`. This is important because it will cause gcc - @# to automatically use avx512 extensions if they're avaialable, which would cause it to break on CPUs - @# that don't support it. - PORTABLE=1 USE_SSE=1 DEBUG_LEVEL=0 USE_RTTI=1 \ - $(MAKE) -C $(dir $@) shared_lib -j${NPROC} - strip --strip-all $@ - - # Cleanup for less disk use / faster CI caching. - rm -rf $(dir $@)/shared-objects - find $(dir $@) -name "*.[oda]" -exec rm -f {} \; - -# Run the protobuf compiler to generate message and gRPC service implementations. -# Invoke protoc with local and third-party include paths set. The `go list` tool -# is used to map submodules to corresponding go.mod versions and paths. -%.pb.go: %.proto ${WORKDIR}/protoc-gen-gogo - PATH=${WORKDIR}:$${PATH} ;\ - protoc -I . $(foreach module, $(PROTOC_INC_MODULES), -I$(module_path)) \ - --gogo_out=paths=source_relative,plugins=grpc:. \ - $*.proto - -%.pb.gw.go: %.proto ${WORKDIR}/protoc-gen-grpc-gateway - PATH=${WORKDIR}:$${PATH} ;\ - protoc -I . $(foreach module, $(PROTOC_INC_MODULES), -I$(module_path)) \ - --grpc-gateway_out=logtostderr=true,paths=source_relative,generate_unbound_methods=false,grpc_api_configuration=$*_gateway.yaml:. \ - $*.proto - -# Rule to build protoc-gen-gogo. -${WORKDIR}/protoc-gen-gogo: - go mod download github.com/golang/protobuf - go build -o $@ github.com/gogo/protobuf/protoc-gen-gogo - -# Rule to build protoc-gen-grpc-gateway. -${WORKDIR}/protoc-gen-grpc-gateway: - go mod download github.com/grpc-ecosystem/grpc-gateway - go build -o $@ github.com/grpc-ecosystem/grpc-gateway/protoc-gen-grpc-gateway - -# Rule for generic go-install-able Go binaries. -${WORKDIR}/go-path/bin/%: go-install - -.PHONY: ci-builder-image as-ci go-install go-test-fast go-test-ci diff --git a/mk/common-config.mk b/mk/common-config.mk deleted file mode 100644 index 9a51f213..00000000 --- a/mk/common-config.mk +++ /dev/null @@ -1,41 +0,0 @@ -# common-config.mk defines build configuration which is common to both -# go.gazette.dev/core, and external consumer application projects -# which re-use the Gazette build infrastructure. - -# Git version & date which are injected into built binaries. -VERSION = $(shell git describe --dirty --tags) -DATE = $(shell date +%F-%T-%Z) -# Number of available processors for parallel builds. -NPROC := $(if ${NPROC},${NPROC},$(shell nproc)) - -# Version of Rocks to build against. -# - This is tightly coupled github.com/tecbot/gorocksdb (update them together). -# - Also update .github/workflows/ci-workflow.yaml -ROCKSDB_VERSION = 6.22.1 - -# Repository root (the directory of the invoked Makefile). -ROOTDIR = $(abspath $(dir $(firstword $(MAKEFILE_LIST)))) -# Location of the gazette/core repository (one level up from common-config.mk) -COREDIR = $(abspath $(dir $(lastword $(MAKEFILE_LIST)))/..) -# Location to place intermediate files and output artifacts -# during the build process. Note the go tool ignores directories -# with leading '.' or '_'. -WORKDIR = ${ROOTDIR}/.build -# Location of RocksDB source under $WORKDIR. -ROCKSDIR = ${WORKDIR}/rocksdb-v${ROCKSDB_VERSION} - -# PROTOC_INC_MODULES is an append-able list of Go modules -# which should be included with `protoc` invocations. -PROTOC_INC_MODULES += "github.com/golang/protobuf" -PROTOC_INC_MODULES += "github.com/gogo/protobuf" - -# module_path expands a $(module), like go.gazette.dev/core, to the local path -# of its respository as currently specified by go.mod. -module_path = $(shell go list -f '{{ .Dir }}' -m $(module)) - -# Export appropriate CGO and run-time linker flags to build, link, -# and run against against our copy of Rocks. -export CGO_CFLAGS = -I${ROCKSDIR}/include -export CGO_CPPFLAGS = -I${ROCKSDIR}/include -export CGO_LDFLAGS = -L${ROCKSDIR} -lrocksdb -lstdc++ -lm -lz -lbz2 -lsnappy -llz4 -lzstd -ldl -export LD_LIBRARY_PATH = ${ROCKSDIR} diff --git a/mk/microk8s.mk b/mk/microk8s.mk deleted file mode 100644 index 4d6a38d0..00000000 --- a/mk/microk8s.mk +++ /dev/null @@ -1,13 +0,0 @@ - -# Running microk8s.reset can take quite a while when many pods are running. -# This command nukes microk8s from orbit, by deleting its Etcd backing store -# and restarting, then spinning up Helm. In practice it's much faster for -# tearing down and re-initializing a local cluster. -microk8s-reset: - sudo microk8s.stop - sudo rm -r /var/snap/microk8s/common/var/run/etcd/member - sleep 1 - sudo microk8s.start || true - microk8s.enable dns storage registry - -.PHONY: microk8s-reset From 3b7bd147c8c37abfd0928c8de16e5f95bf4f2f6a Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Fri, 10 Jan 2025 12:37:09 -0600 Subject: [PATCH 7/9] broker: replace --disable-stores with --file-only Remove --disable-stores behavior, which causes fragments to never persist to their configured remote store. This flag has historically been used in testing contexts. Replace it with --file-only, which achieves a similar testing outcome by re-interpreting a s3://bucket/foobar store to instead use file:///foobar This preserves the desired behavior of keeping changes local and not writing to remote fragment stores, while allowing for test brokers to be restarted without data loss, as well as allowing for testing of features that require persistence (like journal suspension). --- broker/fragment/index.go | 4 ---- broker/fragment/stores.go | 14 ++++++++------ cmd/gazette/main.go | 7 +++++-- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/broker/fragment/index.go b/broker/fragment/index.go index 9f812188..8cdc7ca2 100644 --- a/broker/fragment/index.go +++ b/broker/fragment/index.go @@ -217,10 +217,6 @@ func (fi *Index) Inspect(ctx context.Context, callback func(CoverSet) error) err func WalkAllStores(ctx context.Context, name pb.Journal, stores []pb.FragmentStore) (CoverSet, error) { var set CoverSet - if DisableStores { - return set, nil - } - for _, store := range stores { var err = List(ctx, store, name, func(f pb.Fragment) { set, _ = set.Add(Fragment{Fragment: f}) diff --git a/broker/fragment/stores.go b/broker/fragment/stores.go index 455b63fe..69da56f0 100644 --- a/broker/fragment/stores.go +++ b/broker/fragment/stores.go @@ -17,9 +17,9 @@ import ( pb "go.gazette.dev/core/broker/protocol" ) -// DisableStores disables the use of configured journal stores. -// If true, fragments are not persisted, and stores are not listed for existing fragments. -var DisableStores bool = false +// ForceFileStore reinterprets journal stores to use the file:// scheme, ignoring +// a configured cloud scheme and bucket such as s3:// or gcs:// +var ForceFileStore bool = false type backend interface { Provider() string @@ -48,6 +48,10 @@ var sharedStores = struct { } func getBackend(scheme string) backend { + if ForceFileStore { + return sharedStores.fs + } + switch scheme { case "s3": return sharedStores.s3 @@ -89,9 +93,7 @@ func Open(ctx context.Context, fragment pb.Fragment) (io.ReadCloser, error) { // present, this is a no-op. If the Spool has not been compressed incrementally, // it will be compressed before being persisted. func Persist(ctx context.Context, spool Spool, spec *pb.JournalSpec) error { - if DisableStores { - return nil // No-op. - } else if len(spec.Fragment.Stores) == 0 { + if len(spec.Fragment.Stores) == 0 { return nil // No-op. } spool.BackingStore = spec.Fragment.Stores[0] diff --git a/cmd/gazette/main.go b/cmd/gazette/main.go index fcf13c48..4b7b9beb 100644 --- a/cmd/gazette/main.go +++ b/cmd/gazette/main.go @@ -3,6 +3,7 @@ package main import ( "context" "crypto/tls" + "fmt" "net/http" "os" "os/signal" @@ -32,10 +33,10 @@ var Config = new(struct { mbp.ServiceConfig Limit uint32 `long:"limit" env:"LIMIT" default:"1024" description:"Maximum number of Journals the broker will allocate"` FileRoot string `long:"file-root" env:"FILE_ROOT" description:"Local path which roots file:// fragment stores (optional)"` + FileOnly bool `long:"file-only" env:"FILE_ONLY" description:"Use the local file:// store for all journal fragments, ignoring cloud bucket storage configuration (for example, S3)"` MaxAppendRate uint32 `long:"max-append-rate" env:"MAX_APPEND_RATE" default:"0" description:"Max rate (in bytes-per-sec) that any one journal may be appended to. If zero, there is no max rate"` MaxReplication uint32 `long:"max-replication" env:"MAX_REPLICATION" default:"9" description:"Maximum effective replication of any one journal, which upper-bounds its stated replication."` MinAppendRate uint32 `long:"min-append-rate" env:"MIN_APPEND_RATE" default:"65536" description:"Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to sustain this rate are aborted"` - DisableStores bool `long:"disable-stores" env:"DISABLE_STORES" description:"Disable use of any configured journal fragment stores. The broker will neither list or persist remote fragments, and all data is discarded on broker exit."` WatchDelay time.Duration `long:"watch-delay" env:"WATCH_DELAY" default:"30ms" description:"Delay applied to the application of watched Etcd events. Larger values amortize the processing of fast-changing Etcd keys."` AuthKeys string `long:"auth-keys" env:"AUTH_KEYS" description:"Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all keys) Authorization tokens." json:"-"` AutoSuspend bool `long:"auto-suspend" env:"AUTO_SUSPEND" description:"Automatically suspend journals which have persisted all fragments"` @@ -104,12 +105,14 @@ func (cmdServe) Execute(args []string) error { _, err = os.Stat(Config.Broker.FileRoot) mbp.Must(err, "configured local file:// root failed") fragment.FileSystemStoreRoot = Config.Broker.FileRoot + } else if Config.Broker.FileOnly { + mbp.Must(fmt.Errorf("--file-root is not configured"), "a file root must be defined when using --file-only") } broker.AutoSuspend = Config.Broker.AutoSuspend broker.MaxAppendRate = int64(Config.Broker.MaxAppendRate) broker.MinAppendRate = int64(Config.Broker.MinAppendRate) - fragment.DisableStores = Config.Broker.DisableStores + fragment.ForceFileStore = Config.Broker.FileOnly pb.MaxReplication = int32(Config.Broker.MaxReplication) var ( From efc250e89eb6b4f68192965925cbaabd5242887b Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Sat, 11 Jan 2025 11:25:51 -0600 Subject: [PATCH 8/9] go.mod: fix incorrect v2 go-sqlite package --- consumer/store-sqlite/store.go | 57 ++++++++++++++++++---------------- go.mod | 2 +- go.sum | 4 +-- 3 files changed, 34 insertions(+), 29 deletions(-) diff --git a/consumer/store-sqlite/store.go b/consumer/store-sqlite/store.go index 4a3b8743..380b0fe4 100644 --- a/consumer/store-sqlite/store.go +++ b/consumer/store-sqlite/store.go @@ -21,10 +21,10 @@ import ( "time" "unsafe" + "github.com/jgraettinger/gorocksdb" "github.com/mattn/go-sqlite3" "github.com/pkg/errors" log "github.com/sirupsen/logrus" - "github.com/jgraettinger/gorocksdb" "go.gazette.dev/core/broker/client" "go.gazette.dev/core/consumer" pc "go.gazette.dev/core/consumer/protocol" @@ -168,21 +168,20 @@ func NewStore(recorder *recoverylog.Recorder) (*Store, error) { // The "gazette_checkpoint" table is then created, and any provided statements // are prepared and added to Store.Stmts in the same order as provided to Open. // -// store.Open( -// // Create myTable if it doesn't yet exist: -// `CREATE TABLE IF NOT EXISTS myTable ( -// id BLOB PRIMARY KEY NOT NULL, -// valueOne INTEGER, -// valueTwo TEXT -// );`, -// // Statement for querying on valueOne: -// `SELECT id, valueTwo FROM myTable WHERE valueOne > ?;`, -// // Statement for upserting into myTable: -// `INSERT INTO myTable(id, valueOne, valueTwo) VALUES(:id, :one, :two) -// ON CONFLICT(id) DO UPDATE SET valueOne = valueOne + :one, valueTwo = :two`, -// ) -// // store.Stmts[0] is now prepared for queries, and store.Stmts[1] for upserts. -// +// store.Open( +// // Create myTable if it doesn't yet exist: +// `CREATE TABLE IF NOT EXISTS myTable ( +// id BLOB PRIMARY KEY NOT NULL, +// valueOne INTEGER, +// valueTwo TEXT +// );`, +// // Statement for querying on valueOne: +// `SELECT id, valueTwo FROM myTable WHERE valueOne > ?;`, +// // Statement for upserting into myTable: +// `INSERT INTO myTable(id, valueOne, valueTwo) VALUES(:id, :one, :two) +// ON CONFLICT(id) DO UPDATE SET valueOne = valueOne + :one, valueTwo = :two`, +// ) +// // store.Stmts[0] is now prepared for queries, and store.Stmts[1] for upserts. func (s *Store) Open(bootstrapSQL string, statements ...string) error { // Finish initialization and open page DB. s.PageDBOptions.SetEnv(s.PageDBEnv) @@ -253,8 +252,9 @@ func (s *Store) Open(bootstrapSQL string, statements ...string) error { // database name (which is held constant across Store recoveries) to a URI // specific to this *Store instance and which is suited for ATTACH-ing to the // primary database. Eg: -// URIForDB("sept_2019.db") => -// "file:sept_2019.db?_journal_mode=WAL&other-setting=bar&vfs=store-123456&..." +// +// URIForDB("sept_2019.db") => +// "file:sept_2019.db?_journal_mode=WAL&other-setting=bar&vfs=store-123456&..." func (s *Store) URIForDB(name string) string { return "file:" + name + "?" + s.SQLiteURIValues.Encode() } @@ -339,6 +339,12 @@ func (s *Store) Destroy() { C.recFSFree(s.vfs) } + // Column family handles must be closed before the DB is. + for _, c := range s.PageDBColumns { + c.Destroy() + } + s.PageDBColumns = nil + if s.pages != nil { s.pages.Close() } @@ -362,14 +368,13 @@ func (s *Store) pageDBPath() string { // the linked SQLite library was built with. See https://www.sqlite.org/compile.html // for a full listing. Note the "SQLITE_" prefix is dropped in the returned set: // -// map[string]struct{ -// "COMPILER=gcc-8.3.0": {}, -// "ENABLE_BATCH_ATOMIC_WRITE": {}, -// "ENABLE_COLUMN_METADATA": {}, -// "ENABLE_DBSTAT_VTAB": {}, -// ... etc ... -// } -// +// map[string]struct{ +// "COMPILER=gcc-8.3.0": {}, +// "ENABLE_BATCH_ATOMIC_WRITE": {}, +// "ENABLE_COLUMN_METADATA": {}, +// "ENABLE_DBSTAT_VTAB": {}, +// ... etc ... +// } func SQLiteCompiledOptions() (map[string]struct{}, error) { var db, err = sql.Open("sqlite3", ":memory:") if err != nil { diff --git a/go.mod b/go.mod index ebbe2e30..c4345b7b 100644 --- a/go.mod +++ b/go.mod @@ -29,7 +29,7 @@ require ( github.com/jgraettinger/urkel v0.1.2 github.com/klauspost/compress v1.13.5 github.com/lib/pq v1.10.2 - github.com/mattn/go-sqlite3 v2.0.3+incompatible + github.com/mattn/go-sqlite3 v1.14.24 github.com/olekukonko/tablewriter v0.0.5 github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.11.1 diff --git a/go.sum b/go.sum index c70c9693..1b4e7d31 100644 --- a/go.sum +++ b/go.sum @@ -328,8 +328,8 @@ github.com/mattn/go-ieproxy v0.0.1/go.mod h1:pYabZ6IHcRpFh7vIaLfK7rdcWgFEb3SFJ6/ github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.13 h1:lTGmDsbAYt5DmK6OnoV7EuIF1wEIFAcxld6ypU4OSgU= github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= -github.com/mattn/go-sqlite3 v2.0.3+incompatible h1:gXHsfypPkaMZrKbD5209QV9jbUTJKjyR5WD3HYQSd+U= -github.com/mattn/go-sqlite3 v2.0.3+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= +github.com/mattn/go-sqlite3 v1.14.24 h1:tpSp2G2KyMnnQu99ngJ47EIkWVmliIizyZBfPrBWDRM= +github.com/mattn/go-sqlite3 v1.14.24/go.mod h1:Uh1q+B4BYcTPb+yiD3kU8Ct7aC0hY9fxUwlHK0RXw+Y= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= From 95692a27b46f74adfc024d3d82f367aebc804442 Mon Sep 17 00:00:00 2001 From: Johnny Graettinger Date: Thu, 9 Jan 2025 14:26:29 -0600 Subject: [PATCH 9/9] docs: update overview-build-and-test and regenerate reference docs --- docs/_static/cmd-gazctl-attach-uuids.txt | 15 +--- docs/_static/cmd-gazctl-journals-append.txt | 39 ++++---- docs/_static/cmd-gazctl-journals-apply.txt | 37 ++++---- docs/_static/cmd-gazctl-journals-edit.txt | 35 +++++--- .../_static/cmd-gazctl-journals-fragments.txt | 39 ++++---- docs/_static/cmd-gazctl-journals-list.txt | 44 +++++---- docs/_static/cmd-gazctl-journals-prune.txt | 40 +++++---- docs/_static/cmd-gazctl-journals-read.txt | 44 +++++---- .../cmd-gazctl-journals-reset-head.txt | 31 ++++--- docs/_static/cmd-gazctl-journals-suspend.txt | 45 ++++++++++ docs/_static/cmd-gazctl-print-config.txt | 9 +- docs/_static/cmd-gazctl-shards-apply.txt | 50 +++++++---- docs/_static/cmd-gazctl-shards-edit.txt | 48 ++++++---- docs/_static/cmd-gazctl-shards-list.txt | 56 +++++++----- docs/_static/cmd-gazctl-shards-prune.txt | 51 ++++++++--- docs/_static/cmd-gazctl.txt | 9 +- docs/_static/cmd-gazette-print-config.txt | 63 +++++++++---- docs/_static/cmd-gazette-serve.txt | 63 +++++++++---- docs/overview-build-and-test.rst | 90 ++----------------- docs/reference-gazctl.rst | 4 + 20 files changed, 477 insertions(+), 335 deletions(-) create mode 100644 docs/_static/cmd-gazctl-journals-suspend.txt diff --git a/docs/_static/cmd-gazctl-attach-uuids.txt b/docs/_static/cmd-gazctl-attach-uuids.txt index 775a1dce..ebeeeadd 100644 --- a/docs/_static/cmd-gazctl-attach-uuids.txt +++ b/docs/_static/cmd-gazctl-attach-uuids.txt @@ -42,18 +42,11 @@ Optionally compose with "jq" to un-nest the JSON objects: > | jq -c '{uuid: .uuid} + .record' -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [attach-uuids command options] - --template= Go text/template for output (default: "{{.UUID}},{{.Line}}\n") - --max-length= Maximum allowed byte-length of an input line (default: 4194304) - --skip-header Omit the first line of each input file + --template= Go text/template for output (default: "{{.UUID}},{{.Line}}\n") + --max-length= Maximum allowed byte-length of an input line (default: 4194304) + --skip-header Omit the first line of each input file diff --git a/docs/_static/cmd-gazctl-journals-append.txt b/docs/_static/cmd-gazctl-journals-append.txt index 84eccdf2..2b191b1e 100644 --- a/docs/_static/cmd-gazctl-journals-append.txt +++ b/docs/_static/cmd-gazctl-journals-append.txt @@ -53,27 +53,34 @@ cat /dev/stdout > my-fifo & # Hold my-fifo open so gazctl doesn't read EOF. gazctl journals append -l my-label --framing 'lines' --mapping 'rendezvous' --input my-fifo -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [append command options] - -l, --selector= Label selector of journals to append to - -i, --input= Input file path. Use '-' for stdin (default: -) - -f, --framing=[none|lines|fixed] Framing of records in input, if any (default: none) - -m, --mapping=[random|modulo|rendezvous|direct] Mapping function of records to journals (default: random) - --base64 Partition keys under 'lines' framing are interpreted as base64 + -l, --selector= Label selector of journals to append to + -i, --input= Input file path. Use '-' for stdin (default: -) + -f, --framing=[none|lines|fixed] Framing of records in input, if any (default: none) + -m, --mapping=[random|modulo|rendezvous|direct] Mapping function of records to journals (default: random) + --base64 Partition keys under 'lines' framing are interpreted as base64 diff --git a/docs/_static/cmd-gazctl-journals-apply.txt b/docs/_static/cmd-gazctl-journals-apply.txt index d81de908..bd940693 100644 --- a/docs/_static/cmd-gazctl-journals-apply.txt +++ b/docs/_static/cmd-gazctl-journals-apply.txt @@ -35,26 +35,33 @@ gazctl supports a flag which will send changes in batches of at most a single Etcd transaction and it should therefore be used with caution. If possible, prefer to use label selectors to limit the number of changes. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [apply command options] - --specs= Input specifications path to apply. Use '-' for stdin (default: -) - --dry-run Perform a dry-run of the apply - --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, all changes are issued in a single transaction - (default: 0) + --specs= Input specifications path to apply. Use '-' for stdin (default: -) + --dry-run Perform a dry-run of the apply + --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, + all changes are issued in a single transaction (default: 0) diff --git a/docs/_static/cmd-gazctl-journals-edit.txt b/docs/_static/cmd-gazctl-journals-edit.txt index 130b1b94..5e3d6e6e 100644 --- a/docs/_static/cmd-gazctl-journals-edit.txt +++ b/docs/_static/cmd-gazctl-journals-edit.txt @@ -29,25 +29,32 @@ gazctl supports a flag which will send changes in batches of at most a single Etcd transaction and it should therefore be used with caution. If possible, prefer to use label selectors to limit the number of changes. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [edit command options] - -l, --selector= Label Selector query to filter on - --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, all changes are issued in a single transaction - (default: 0) + -l, --selector= Label Selector query to filter on + --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, + all changes are issued in a single transaction (default: 0) diff --git a/docs/_static/cmd-gazctl-journals-fragments.txt b/docs/_static/cmd-gazctl-journals-fragments.txt index 846e4305..388e8337 100644 --- a/docs/_static/cmd-gazctl-journals-fragments.txt +++ b/docs/_static/cmd-gazctl-journals-fragments.txt @@ -51,27 +51,34 @@ gazctl journals fragments -l name=my/journal --url-ttl 1m --from $(date -d "1 ho gazctl journals fragments -l my-label --format json --url-ttl 1h --from $(date -d 3AM '+%s') --to $(date -d 4:05AM '+%s') --format json -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [fragments command options] - -l, --selector= Label Selector query to filter on - -o, --format=[table|json|proto] Output format (default: table) - --from= Restrict to fragments created at or after this time, in unix seconds since epoch - --to= Restrict to fragments created before this time, in unix seconds since epoch - --url-ttl= Provide a signed GET URL with the given TTL + -l, --selector= Label Selector query to filter on + -o, --format=[table|json|proto] Output format (default: table) + --from= Restrict to fragments created at or after this time, in unix seconds since epoch + --to= Restrict to fragments created before this time, in unix seconds since epoch + --url-ttl= Provide a signed GET URL with the given TTL (default: 0s) diff --git a/docs/_static/cmd-gazctl-journals-list.txt b/docs/_static/cmd-gazctl-journals-list.txt index 5b23e646..4ed065ed 100644 --- a/docs/_static/cmd-gazctl-journals-list.txt +++ b/docs/_static/cmd-gazctl-journals-list.txt @@ -24,29 +24,37 @@ typically, common configuration. This hierarchy is simply sugar for and is exactly equivalent to the original JournalSpecs. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [list command options] - -l, --selector= Label Selector query to filter on - -o, --format=[table|yaml|json|proto] Output format (default: table) - -L, --label-columns= Labels to present as columns, eg -L label-one -L label-two - -p, --primary Show primary column - -r, --replicas Show replicas column - --rf Show replication factor column - --stores Show fragment store column + -l, --selector= Label Selector query to filter on + -o, --format=[table|yaml|json|proto] Output format (default: table) + -L, --label-columns= Labels to present as columns, eg -L label-one -L label-two + -p, --primary Show primary column + -r, --replicas Show replicas column + --rf Show replication factor column + --stores Show fragment store column + --watch Use a long-lived watch diff --git a/docs/_static/cmd-gazctl-journals-prune.txt b/docs/_static/cmd-gazctl-journals-prune.txt index ff7f352e..bb3eabdd 100644 --- a/docs/_static/cmd-gazctl-journals-prune.txt +++ b/docs/_static/cmd-gazctl-journals-prune.txt @@ -3,32 +3,40 @@ Usage: Deletes fragments across all configured fragment stores of matching journals that are older than the configured retention. -There is a caveat when pruning journals. For a given journal, there could be multiple fragments covering the same offset. These fragments contain identical data at a given -offset, but the brokers are tracking only the largest fragment, i.e. the fragment that covers the largest span of offsets. As a result, the prune command will delete only this -tracked fragment, leaving the smaller fragments untouched. As a workaround, operators can wait for the fragment listing to refresh and prune the journals again. +There is a caveat when pruning journals. For a given journal, there could be multiple fragments covering the same offset. These fragments contain +identical data at a given offset, but the brokers are tracking only the largest fragment, i.e. the fragment that covers the largest span of +offsets. As a result, the prune command will delete only this tracked fragment, leaving the smaller fragments untouched. As a workaround, +operators can wait for the fragment listing to refresh and prune the journals again. Use --selector to supply a LabelSelector to select journals to prune. See "journals list --help" for details and examples. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [prune command options] - -l, --selector= Label Selector query to filter on - --dry-run Perform a dry-run of the apply + -l, --selector= Label Selector query to filter on + --dry-run Perform a dry-run of the apply diff --git a/docs/_static/cmd-gazctl-journals-read.txt b/docs/_static/cmd-gazctl-journals-read.txt index 1affd294..a0f0fdbd 100644 --- a/docs/_static/cmd-gazctl-journals-read.txt +++ b/docs/_static/cmd-gazctl-journals-read.txt @@ -58,29 +58,37 @@ echo "{}" > offsets.json # Must already exist. gazctl journals read -l my-label -o output --offsets offsets.json --offsets-out offsets.json --broker.cache.size=256 --zone=us-east-1 -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [read command options] - -l, --selector= Label selector of journals to read - -b, --block Do not exit on journal EOF; wait for new data until signaled - --tail Start reading from the journal write-head (rather than offset 0) - -o, --output= Output file path. Use '-' for stdout (default: -) - --offsets= Path from which initial journal offsets are read at startup - --offsets-out= Path to which final journal offsets are written at exit - --file-root= Filesystem path which roots file:// fragment store + -l, --selector= Label selector of journals to read + -b, --block Do not exit on journal EOF; wait for new data until signaled + --tail Start reading from the journal write-head (rather than offset 0) + -o, --output= Output file path. Use '-' for stdout (default: -) + --offsets= Path from which initial journal offsets are read at startup + --offsets-out= Path to which final journal offsets are written at exit + --file-root= Filesystem path which roots file:// fragment store + --from= Skip over fragments persisted before this time, in unix seconds since epoch diff --git a/docs/_static/cmd-gazctl-journals-reset-head.txt b/docs/_static/cmd-gazctl-journals-reset-head.txt index ca11d1d3..70672488 100644 --- a/docs/_static/cmd-gazctl-journals-reset-head.txt +++ b/docs/_static/cmd-gazctl-journals-reset-head.txt @@ -37,23 +37,30 @@ reset-head is safe to run against journals which are already consistent and and are being actively appended to. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [journals command options] + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [reset-head command options] - -l, --selector= Label Selector query to filter on + -l, --selector= Label Selector query to filter on diff --git a/docs/_static/cmd-gazctl-journals-suspend.txt b/docs/_static/cmd-gazctl-journals-suspend.txt new file mode 100644 index 00000000..c5169815 --- /dev/null +++ b/docs/_static/cmd-gazctl-journals-suspend.txt @@ -0,0 +1,45 @@ +Usage: + gazctl [OPTIONS] journals [journals-OPTIONS] suspend [suspend-OPTIONS] + +Suspend idle journals to use fewer or zero broker replicas. + +Suspension updates the 'suspend' field of the JournalSpec with its suspended +status and its resumption offset. When applying other updates to JournalSpecs, +operators utilizing journal suspension must take care to not overwrite the +journal's suspension configuration. + +Typically this means reading the current JournalSpec and its ModRevision, +copying the current 'suspend' field alongside other changes being made, +and then applying the updated JournalSpec with ExpectModRevision. + +The 'journals edit' subcommand uses this workflow and is safe to use with suspension. + + +Help Options: + -h, --help Show this help message + +[journals command options] + + Interact with broker journals: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + + Broker: + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + +[suspend command options] + -l, --selector= Label Selector query to filter on + -f, --force Suspend journals even if they have local fragments + diff --git a/docs/_static/cmd-gazctl-print-config.txt b/docs/_static/cmd-gazctl-print-config.txt index 024a846f..4c798129 100644 --- a/docs/_static/cmd-gazctl-print-config.txt +++ b/docs/_static/cmd-gazctl-print-config.txt @@ -5,13 +5,6 @@ print-config parses the combined configuration from gazctl.ini, flags, and environment variables, and then writes the configuration to stdout in INI format. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message diff --git a/docs/_static/cmd-gazctl-shards-apply.txt b/docs/_static/cmd-gazctl-shards-apply.txt index 65398904..092adc0f 100644 --- a/docs/_static/cmd-gazctl-shards-apply.txt +++ b/docs/_static/cmd-gazctl-shards-apply.txt @@ -25,31 +25,45 @@ gazctl supports a flag which will send changes in batches of at most a single Etcd transaction and it should therefore be used with caution. If possible, prefer to use label selectors to limit the number of changes. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [shards command options] + Interact with consumer shards: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Consumer: - --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] - --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$CONSUMER_CACHE_SIZE] - --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] + --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] + --consumer.cert-file= Path to the client TLS certificate [$CONSUMER_CERT_FILE] + --consumer.cert-key-file= Path to the client TLS private key [$CONSUMER_CERT_KEY_FILE] + --consumer.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$CONSUMER_TRUSTED_CA_FILE] + --consumer.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$CONSUMER_AUTH_KEYS] + --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$CONSUMER_CACHE_SIZE] + --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [apply command options] - --specs= Input specifications path to apply. Use '-' for stdin (default: -) - --dry-run Perform a dry-run of the apply - --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, all changes are issued in a single transaction - (default: 0) + --specs= Input specifications path to apply. Use '-' for stdin (default: -) + --dry-run Perform a dry-run of the apply + --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, + all changes are issued in a single transaction (default: 0) diff --git a/docs/_static/cmd-gazctl-shards-edit.txt b/docs/_static/cmd-gazctl-shards-edit.txt index 057c37bf..d37fff1b 100644 --- a/docs/_static/cmd-gazctl-shards-edit.txt +++ b/docs/_static/cmd-gazctl-shards-edit.txt @@ -29,30 +29,44 @@ gazctl supports a flag which will send changes in batches of at most a single Etcd transaction and it should therefore be used with caution. If possible, prefer to use label selectors to limit the number of changes. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [shards command options] + Interact with consumer shards: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Consumer: - --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] - --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$CONSUMER_CACHE_SIZE] - --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] + --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] + --consumer.cert-file= Path to the client TLS certificate [$CONSUMER_CERT_FILE] + --consumer.cert-key-file= Path to the client TLS private key [$CONSUMER_CERT_KEY_FILE] + --consumer.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$CONSUMER_TRUSTED_CA_FILE] + --consumer.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$CONSUMER_AUTH_KEYS] + --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$CONSUMER_CACHE_SIZE] + --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [edit command options] - -l, --selector= Label Selector query to filter on - --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, all changes are issued in a single transaction - (default: 0) + -l, --selector= Label Selector query to filter on + --max-txn-size= maximum number of specs to be processed within an apply transaction. If 0, the default, + all changes are issued in a single transaction (default: 0) diff --git a/docs/_static/cmd-gazctl-shards-list.txt b/docs/_static/cmd-gazctl-shards-list.txt index 452e306f..fd46b5b7 100644 --- a/docs/_static/cmd-gazctl-shards-list.txt +++ b/docs/_static/cmd-gazctl-shards-list.txt @@ -19,34 +19,48 @@ It's recommended that --lag be used with a relatively focused --selector, as fetching consumption lag for a large number of shards may take a while. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message [shards command options] + Interact with consumer shards: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Consumer: - --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] - --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$CONSUMER_CACHE_SIZE] - --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] + --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] + --consumer.cert-file= Path to the client TLS certificate [$CONSUMER_CERT_FILE] + --consumer.cert-key-file= Path to the client TLS private key [$CONSUMER_CERT_KEY_FILE] + --consumer.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$CONSUMER_TRUSTED_CA_FILE] + --consumer.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$CONSUMER_AUTH_KEYS] + --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$CONSUMER_CACHE_SIZE] + --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [list command options] - -l, --selector= Label Selector query to filter on - -o, --format=[table|yaml|json|proto] Output format (default: table) - -L, --label-columns= Labels to present as columns, eg -L label-one -L label-two - -p, --primary Show primary column - -r, --replicas Show replicas column - --rf Show replication factor column - --lag Show the amount of unread data for each shard + -l, --selector= Label Selector query to filter on + -o, --format=[table|yaml|json|proto] Output format (default: table) + -L, --label-columns= Labels to present as columns, eg -L label-one -L label-two + -p, --primary Show primary column + -r, --replicas Show replicas column + --rf Show replication factor column + --lag Show the amount of unread data for each shard diff --git a/docs/_static/cmd-gazctl-shards-prune.txt b/docs/_static/cmd-gazctl-shards-prune.txt index 8a382105..3e363f34 100644 --- a/docs/_static/cmd-gazctl-shards-prune.txt +++ b/docs/_static/cmd-gazctl-shards-prune.txt @@ -10,30 +10,53 @@ Prune log examines the provided hints to identify Fragments of the log which have no intersection with any live files of the DB, and can thus be safely deleted. +CAUTION: -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] +When pruning recovery logs which have been forked from other logs, +it's crucial that *all* shards which participate in the forked log +history are included in the prune operation. When pruning a log, +hints from all referencing shards are inspected to determine if a +fragment is overlapped, and a failure to include a shard which +references the log may cause data it depends on to be deleted. -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] Help Options: - -h, --help Show this help message + -h, --help Show this help message [shards command options] + Interact with consumer shards: + --zone= Availability zone within which this process is running (default: local) [$ZONE] + + Logging: + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + Consumer: - --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] - --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$CONSUMER_CACHE_SIZE] - --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] + --consumer.address= Service address endpoint (default: http://localhost:8080) [$CONSUMER_ADDRESS] + --consumer.cert-file= Path to the client TLS certificate [$CONSUMER_CERT_FILE] + --consumer.cert-key-file= Path to the client TLS private key [$CONSUMER_CERT_KEY_FILE] + --consumer.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$CONSUMER_TRUSTED_CA_FILE] + --consumer.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$CONSUMER_AUTH_KEYS] + --consumer.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$CONSUMER_CACHE_SIZE] + --consumer.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$CONSUMER_CACHE_TTL] Broker: - --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] - --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) (default: 0) [$BROKER_CACHE_SIZE] - --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] + --broker.address= Service address endpoint (default: http://localhost:8080) [$BROKER_ADDRESS] + --broker.cert-file= Path to the client TLS certificate [$BROKER_CERT_FILE] + --broker.cert-key-file= Path to the client TLS private key [$BROKER_CERT_KEY_FILE] + --broker.trusted-ca-file= Path to the trusted CA for client verification of server certificates + [$BROKER_TRUSTED_CA_FILE] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys. The first key is used to sign + Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.cache.size= Size of client route cache. If <= zero, no cache is used (server always proxies) + (default: 0) [$BROKER_CACHE_SIZE] + --broker.cache.ttl= Time-to-live of route cache entries. (default: 1m) [$BROKER_CACHE_TTL] [prune command options] - -l, --selector= Label Selector query to filter on - --dry-run Perform a dry-run of the apply + -l, --selector= Label Selector query to filter on + --dry-run Perform a dry-run of the apply diff --git a/docs/_static/cmd-gazctl.txt b/docs/_static/cmd-gazctl.txt index 1c355c2d..451b545e 100644 --- a/docs/_static/cmd-gazctl.txt +++ b/docs/_static/cmd-gazctl.txt @@ -9,15 +9,8 @@ or with '~/.config/gazette/gazctl.ini'. Use the 'print-config' sub-command to in the tool's current configuration. -Application Options: - --zone= Availability zone within which this process is running (default: local) [$ZONE] - -Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] - Help Options: - -h, --help Show this help message + -h, --help Show this help message Available commands: attach-uuids Generate and attach UUIDs to text input records diff --git a/docs/_static/cmd-gazette-print-config.txt b/docs/_static/cmd-gazette-print-config.txt index 3f3e8d6d..13257f3b 100644 --- a/docs/_static/cmd-gazette-print-config.txt +++ b/docs/_static/cmd-gazette-print-config.txt @@ -7,25 +7,58 @@ its responsible journals and will exit only when it can safely do so. Broker: - --broker.zone= Availability zone within which this process is running (default: local) [$BROKER_ZONE] - --broker.id= Unique ID of this process. Auto-generated if not set [$BROKER_ID] - --broker.host= Addressable, advertised hostname or IP of this process. Hostname is used if not set [$BROKER_HOST] - --broker.port= Service port for HTTP and gRPC requests. A random port is used if not set [$BROKER_PORT] - --broker.limit= Maximum number of Journals the broker will allocate (default: 1024) [$BROKER_LIMIT] - --broker.file-root= Local path which roots file:// fragment stores (optional) [$BROKER_FILE_ROOT] - --broker.max-append-rate= Max rate (in bytes-per-sec) that any one journal may be appended to. If zero, there is no max rate (default: 0) [$BROKER_MAX_APPEND_RATE] - --broker.min-append-rate= Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to sustain this rate are aborted (default: 65536) - [$BROKER_MIN_APPEND_RATE] + --broker.zone= Availability zone within which this process is running (default: local) [$BROKER_ZONE] + --broker.id= Unique ID of this process. Auto-generated if not set [$BROKER_ID] + --broker.host= Addressable, advertised hostname or IP of this process. Hostname is used if not set + [$BROKER_HOST] + --broker.port= Service port for HTTP and gRPC requests. A random port is used if not set. Port may also + take the form 'unix:///path/to/socket' to use a Unix Domain Socket [$BROKER_PORT] + --broker.server-cert-file= Path to the server TLS certificate. This option toggles whether TLS is used. If absent, all + other TLS settings are ignored. [$BROKER_SERVER_CERT_FILE] + --broker.server-cert-key-file= Path to the server TLS private key [$BROKER_SERVER_CERT_KEY_FILE] + --broker.server-ca-file= Path to the trusted CA for server verification of client certificates. When present, client + certificates are required and verified against this CA. When absent, client certificates are + not required but are verified against the system CA pool if presented. + [$BROKER_SERVER_CA_FILE] + --broker.peer-cert-file= Path to the client TLS certificate for peer-to-peer requests [$BROKER_PEER_CERT_FILE] + --broker.peer-cert-key-file= Path to the client TLS private key for peer-to-peer requests [$BROKER_PEER_CERT_KEY_FILE] + --broker.peer-ca-file= Path to the trusted CA for client verification of peer server certificates. When absent, the + system CA pool is used instead. [$BROKER_PEER_CA_FILE] + --broker.max-grpc-recv-size= Maximum size of gRPC messages accepted by this server, in bytes (default: 4194304) + [$BROKER_MAX_GRPC_RECV_SIZE] + --broker.allow-origin= Origin to allow in CORS contexts [$BROKER_ALLOW_ORIGIN] + --broker.limit= Maximum number of Journals the broker will allocate (default: 1024) [$BROKER_LIMIT] + --broker.file-root= Local path which roots file:// fragment stores (optional) [$BROKER_FILE_ROOT] + --broker.file-only Use the local file:// store for all journal fragments, ignoring cloud bucket storage + configuration (for example, S3) [$BROKER_FILE_ONLY] + --broker.max-append-rate= Max rate (in bytes-per-sec) that any one journal may be appended to. If zero, there is no + max rate (default: 0) [$BROKER_MAX_APPEND_RATE] + --broker.max-replication= Maximum effective replication of any one journal, which upper-bounds its stated replication. + (default: 9) [$BROKER_MAX_REPLICATION] + --broker.min-append-rate= Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to + sustain this rate are aborted (default: 65536) [$BROKER_MIN_APPEND_RATE] + --broker.watch-delay= Delay applied to the application of watched Etcd events. Larger values amortize the + processing of fast-changing Etcd keys. (default: 30ms) [$BROKER_WATCH_DELAY] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all + keys) Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.auto-suspend Automatically suspend journals which have persisted all fragments [$BROKER_AUTO_SUSPEND] Etcd: - --etcd.address= Etcd service address endpoint (default: http://localhost:2379) [$ETCD_ADDRESS] - --etcd.lease= Time-to-live of Etcd lease (default: 20s) [$ETCD_LEASE_TTL] - --etcd.prefix= Etcd base prefix for broker state and coordination (default: /gazette/cluster) [$ETCD_PREFIX] + --etcd.address= Etcd service address endpoint (default: http://localhost:2379) [$ETCD_ADDRESS] + --etcd.cert-file= Path to the client TLS certificate [$ETCD_CERT_FILE] + --etcd.cert-key-file= Path to the client TLS private key [$ETCD_CERT_KEY_FILE] + --etcd.trusted-ca-file= Path to the trusted CA for client verification of server certificates [$ETCD_TRUSTED_CA_FILE] + --etcd.lease= Time-to-live of Etcd lease (default: 20s) [$ETCD_LEASE_TTL] + --etcd.prefix= Etcd base prefix for broker state and coordination (default: /gazette/cluster) [$ETCD_PREFIX] Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + +Debug: + --debug.port= Port for diagnostics [$DEBUG_PORT] + --debug.private When true, don't serve diagnostics over the main service port [$DEBUG_PRIVATE] Help Options: - -h, --help Show this help message + -h, --help Show this help message diff --git a/docs/_static/cmd-gazette-serve.txt b/docs/_static/cmd-gazette-serve.txt index 3f3e8d6d..13257f3b 100644 --- a/docs/_static/cmd-gazette-serve.txt +++ b/docs/_static/cmd-gazette-serve.txt @@ -7,25 +7,58 @@ its responsible journals and will exit only when it can safely do so. Broker: - --broker.zone= Availability zone within which this process is running (default: local) [$BROKER_ZONE] - --broker.id= Unique ID of this process. Auto-generated if not set [$BROKER_ID] - --broker.host= Addressable, advertised hostname or IP of this process. Hostname is used if not set [$BROKER_HOST] - --broker.port= Service port for HTTP and gRPC requests. A random port is used if not set [$BROKER_PORT] - --broker.limit= Maximum number of Journals the broker will allocate (default: 1024) [$BROKER_LIMIT] - --broker.file-root= Local path which roots file:// fragment stores (optional) [$BROKER_FILE_ROOT] - --broker.max-append-rate= Max rate (in bytes-per-sec) that any one journal may be appended to. If zero, there is no max rate (default: 0) [$BROKER_MAX_APPEND_RATE] - --broker.min-append-rate= Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to sustain this rate are aborted (default: 65536) - [$BROKER_MIN_APPEND_RATE] + --broker.zone= Availability zone within which this process is running (default: local) [$BROKER_ZONE] + --broker.id= Unique ID of this process. Auto-generated if not set [$BROKER_ID] + --broker.host= Addressable, advertised hostname or IP of this process. Hostname is used if not set + [$BROKER_HOST] + --broker.port= Service port for HTTP and gRPC requests. A random port is used if not set. Port may also + take the form 'unix:///path/to/socket' to use a Unix Domain Socket [$BROKER_PORT] + --broker.server-cert-file= Path to the server TLS certificate. This option toggles whether TLS is used. If absent, all + other TLS settings are ignored. [$BROKER_SERVER_CERT_FILE] + --broker.server-cert-key-file= Path to the server TLS private key [$BROKER_SERVER_CERT_KEY_FILE] + --broker.server-ca-file= Path to the trusted CA for server verification of client certificates. When present, client + certificates are required and verified against this CA. When absent, client certificates are + not required but are verified against the system CA pool if presented. + [$BROKER_SERVER_CA_FILE] + --broker.peer-cert-file= Path to the client TLS certificate for peer-to-peer requests [$BROKER_PEER_CERT_FILE] + --broker.peer-cert-key-file= Path to the client TLS private key for peer-to-peer requests [$BROKER_PEER_CERT_KEY_FILE] + --broker.peer-ca-file= Path to the trusted CA for client verification of peer server certificates. When absent, the + system CA pool is used instead. [$BROKER_PEER_CA_FILE] + --broker.max-grpc-recv-size= Maximum size of gRPC messages accepted by this server, in bytes (default: 4194304) + [$BROKER_MAX_GRPC_RECV_SIZE] + --broker.allow-origin= Origin to allow in CORS contexts [$BROKER_ALLOW_ORIGIN] + --broker.limit= Maximum number of Journals the broker will allocate (default: 1024) [$BROKER_LIMIT] + --broker.file-root= Local path which roots file:// fragment stores (optional) [$BROKER_FILE_ROOT] + --broker.file-only Use the local file:// store for all journal fragments, ignoring cloud bucket storage + configuration (for example, S3) [$BROKER_FILE_ONLY] + --broker.max-append-rate= Max rate (in bytes-per-sec) that any one journal may be appended to. If zero, there is no + max rate (default: 0) [$BROKER_MAX_APPEND_RATE] + --broker.max-replication= Maximum effective replication of any one journal, which upper-bounds its stated replication. + (default: 9) [$BROKER_MAX_REPLICATION] + --broker.min-append-rate= Min rate (in bytes-per-sec) at which a client may stream Append RPC content. RPCs unable to + sustain this rate are aborted (default: 65536) [$BROKER_MIN_APPEND_RATE] + --broker.watch-delay= Delay applied to the application of watched Etcd events. Larger values amortize the + processing of fast-changing Etcd keys. (default: 30ms) [$BROKER_WATCH_DELAY] + --broker.auth-keys= Whitespace or comma separated, base64-encoded keys used to sign (first key) and verify (all + keys) Authorization tokens. [$BROKER_AUTH_KEYS] + --broker.auto-suspend Automatically suspend journals which have persisted all fragments [$BROKER_AUTO_SUSPEND] Etcd: - --etcd.address= Etcd service address endpoint (default: http://localhost:2379) [$ETCD_ADDRESS] - --etcd.lease= Time-to-live of Etcd lease (default: 20s) [$ETCD_LEASE_TTL] - --etcd.prefix= Etcd base prefix for broker state and coordination (default: /gazette/cluster) [$ETCD_PREFIX] + --etcd.address= Etcd service address endpoint (default: http://localhost:2379) [$ETCD_ADDRESS] + --etcd.cert-file= Path to the client TLS certificate [$ETCD_CERT_FILE] + --etcd.cert-key-file= Path to the client TLS private key [$ETCD_CERT_KEY_FILE] + --etcd.trusted-ca-file= Path to the trusted CA for client verification of server certificates [$ETCD_TRUSTED_CA_FILE] + --etcd.lease= Time-to-live of Etcd lease (default: 20s) [$ETCD_LEASE_TTL] + --etcd.prefix= Etcd base prefix for broker state and coordination (default: /gazette/cluster) [$ETCD_PREFIX] Logging: - --log.level=[info|debug|warn] Logging level (default: info) [$LOG_LEVEL] - --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + --log.level=[trace|debug|info|warn|error|fatal] Logging level (default: warn) [$LOG_LEVEL] + --log.format=[json|text|color] Logging output format (default: text) [$LOG_FORMAT] + +Debug: + --debug.port= Port for diagnostics [$DEBUG_PORT] + --debug.private When true, don't serve diagnostics over the main service port [$DEBUG_PRIVATE] Help Options: - -h, --help Show this help message + -h, --help Show this help message diff --git a/docs/overview-build-and-test.rst b/docs/overview-build-and-test.rst index da66a0c5..107745f1 100644 --- a/docs/overview-build-and-test.rst +++ b/docs/overview-build-and-test.rst @@ -5,8 +5,6 @@ Most binaries and packages are "pure" Go and can be directly go-installed and go .. code-block:: console - $ export GO111MODULE=on - $ go install go.gazette.dev/core/cmd/gazette $ go install go.gazette.dev/core/cmd/gazctl $ go test go.gazette.dev/core/broker/... @@ -14,8 +12,8 @@ Most binaries and packages are "pure" Go and can be directly go-installed and go Certain packages used by consumer applications, like ``go.gazette.dev/core/consumer/store-rocksdb``, require CGO to build and also require appropriate development libraries for RocksDB. -Standard Linux packages are insufficient, as run-time type information must be enabled -in the RocksDB build (and it's turned off in the standard Debian package, for example). +On Debian and Ubuntu, the ``librocksdb-dev`` and ``libsqlite3-dev`` packages are sufficient, +paired with the ``libsqlite3`` Go build tag. Continuous Integration @@ -24,93 +22,19 @@ Continuous Integration Gazette uses a :githubsource:`Make-based build system` which pulls down and stages development dependencies into a ``.build`` sub-directory of the repository root. -The Make build system offers fully hermetic builds using a Docker-in-Docker -builder. Run CI tests in a hermetic build environment: +Run CI tests as: .. code-block:: console - $ make as-ci target=go-test-ci + $ make go-test-ci Continuous integration builds of Gazette run tests 15 times, with race detection enabled. -Package release Docker images for the Gazette broker and examples, -as ``gazette/broker:latest`` and ``gazette/examples:latest``. - -.. code-block:: console - - $ make as-ci target=ci-release-gazette-broker - $ make as-ci target=ci-release-gazette-examples - -Deploy Gazette's continuous soak test to a Kubernetes cluster (which can be -Docker for Desktop or Minikube). Soak tests run with ``latest`` images. - -.. code-block:: console - - $ kubectl apply -k ./kustomize/test/deploy-stream-sum-with-crash-tests/ - -Push images to a registry. If ``REGISTRY`` is not specified, it defaults to ``localhost:32000`` (which is used by MicroK8s): - -.. code-block:: console - - $ make push-to-registry REGISTRY=my.registry.com - -The ``kustomize`` directory also has a -:githubsource:`helper manifest` for using -a local registry (eg, for development builds). - - -Dependencies -------------- - -It's recommended to use the docker-based build targets described above in most situations, since the -docker image will have all the required dependencies. If you want to execute build targets directly -instead of using ``as-ci``, then the following dependencies are required: +A ``go-test-fast`` target is also available, wich runs tests once without race detection. -* Compression library development headers ("-dev" packages): - - * ``libbz2`` - * ``libsnappy`` - * ``liblz4`` - * ``libzstd`` - * ``libbz2`` - -* Protobuf compiler: - - * ``libprotobuf`` (development headers) - * ``protobuf-compiler`` - -* Etcd: The deb/rpm packages provided in many repositories are likely too old to work with - Gazette. Gazette requires version 3.3 or later. Version 3.4.x is recommended, since that is used - in Gazette CI. - -* Sqlite - - * ``libsqlite3`` (development headers) - * It's also probably useful to have the sqlite3 CLI for debugging - -* RocksDB: On linux systems, this will be downloaded and built automatically. You'll need to have a - few things in order for this to work. Most systems will already have this stuff, but it's listed - here anyway just for the sake of being thorough - - * A C compiler toolchain (on debian-based distros, the ``build-essential`` package will have you covered) - * ``curl`` - * ``ca-certificates`` (so that curl can validate the certificate of the rocksdb download server) - * ``tar`` - -Other Build Targets --------------------- - -If you execute these targets directly, then you'll need to have all of the above dependencies installed. - -.. code-block:: console - - $ make go-install - $ make go-test-fast - - -.. code-block:: console +Build release binaries for Gazette broker and examples using +``go-build`` or ``go-build-arm64`` targets. - $ make go-test-ci Building the Docs ------------------ diff --git a/docs/reference-gazctl.rst b/docs/reference-gazctl.rst index e4125f18..56a07222 100644 --- a/docs/reference-gazctl.rst +++ b/docs/reference-gazctl.rst @@ -41,6 +41,10 @@ gazctl journals reset-head --------------------------- .. literalinclude:: _static/cmd-gazctl-journals-reset-head.txt +gazctl journals suspend +--------------------------- +.. literalinclude:: _static/cmd-gazctl-journals-suspend.txt + gazctl print-config --------------------------- .. literalinclude:: _static/cmd-gazctl-print-config.txt