From 2b646ccf2144d2dfd4f81f3f210aef327d448266 Mon Sep 17 00:00:00 2001 From: Michal-Leszczynski <74614433+Michal-Leszczynski@users.noreply.github.com> Date: Mon, 4 Nov 2024 14:46:13 +0100 Subject: [PATCH] restore: add and fill host info in restore progress (#4088) Restore: add and fill host info in restore progress * chore(go.mod): remove replace directive to SM submodules It was a left-over after feature development:/ * chore(go.mod): bump SM submodules deps * feat(schema): add shard cnt to restore_run_progress It's going to be needed for calculating per shard download/stream bandwidth in progress command. * feat(restore): add and fill shard cnt in restore run progress This commit also moves host shard info to the tablesWorker, as it is commonly reused during restore procedure. * feat(restore): add and fill host info in progress This allows to calculate download/stream per shard bandwidth in 'sctool progress' display. * feat(managerclient): display bandwidth in sctool progress Fixes #4042 * feat(managerclient): include B or iB in SizeSuffix display It is nicer to see: "Size: 10B" instead of "Size: 10" or "Size: 20KiB" instead of "Size: 20k". --- go.mod | 9 +-- go.sum | 6 ++ pkg/schema/table/table.go | 1 + pkg/service/restore/model.go | 42 ++++-------- pkg/service/restore/progress.go | 25 +++++-- pkg/service/restore/service.go | 2 +- pkg/service/restore/tables_worker.go | 52 +++++++------- pkg/service/restore/tablesdir_worker.go | 1 + schema/v3.4.0.cql | 4 +- v3/pkg/managerclient/go.mod | 4 +- v3/pkg/managerclient/go.sum | 8 +-- v3/pkg/managerclient/model.go | 68 ++++++++++++++++++- v3/pkg/managerclient/sizesuffix.go | 14 ++-- v3/pkg/managerclient/utils.go | 6 +- .../v3/pkg/managerclient/LICENSE | 62 +++++++++++++++++ .../v3/pkg/managerclient/model.go | 64 +++++++++++++---- .../scylla-manager/v3/pkg/util/LICENSE | 62 +++++++++++++++++ .../scylla-manager/v3/swagger/LICENSE | 62 +++++++++++++++++ .../models/restore_host_progress.go | 58 ++++++++++++++++ .../scylla-manager/models/restore_progress.go | 32 +++++++++ vendor/modules.txt | 9 +-- 21 files changed, 486 insertions(+), 105 deletions(-) create mode 100644 vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/LICENSE create mode 100644 vendor/github.com/scylladb/scylla-manager/v3/pkg/util/LICENSE create mode 100644 vendor/github.com/scylladb/scylla-manager/v3/swagger/LICENSE create mode 100644 vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_host_progress.go diff --git a/go.mod b/go.mod index 71dbcbfddc..4d9f668c4e 100644 --- a/go.mod +++ b/go.mod @@ -29,9 +29,9 @@ require ( github.com/scylladb/go-reflectx v1.0.1 github.com/scylladb/go-set v1.0.2 github.com/scylladb/gocqlx/v2 v2.8.0 - github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241028110806-78e39cceec83 - github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241028110806-78e39cceec83 - github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241028110806-78e39cceec83 + github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241030073626-e409ae491c83 + github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83 + github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83 github.com/spf13/cobra v1.8.0 github.com/spf13/pflag v1.0.5 github.com/stoewer/go-strcase v1.3.0 @@ -128,8 +128,5 @@ require ( replace ( github.com/gocql/gocql => github.com/scylladb/gocql v1.12.0 github.com/rclone/rclone => github.com/scylladb/rclone v1.54.1-0.20240312172628-afe1fd2aa65e - github.com/scylladb/scylla-manager/v3/pkg/managerclient => ./v3/pkg/managerclient - github.com/scylladb/scylla-manager/v3/pkg/util => ./v3/pkg/util - github.com/scylladb/scylla-manager/v3/swagger => ./v3/swagger google.golang.org/api v0.114.0 => github.com/scylladb/google-api-go-client v0.34.1-patched ) diff --git a/go.sum b/go.sum index d450d949e5..b08afc90f6 100644 --- a/go.sum +++ b/go.sum @@ -1053,6 +1053,12 @@ github.com/scylladb/google-api-go-client v0.34.1-patched h1:DW+T0HA+74o6FDr3TFzV github.com/scylladb/google-api-go-client v0.34.1-patched/go.mod h1:RriRmS2wJXH+2yd9PRTEcR380U9AXmurWwznqVhzsSc= github.com/scylladb/rclone v1.54.1-0.20240312172628-afe1fd2aa65e h1:lJRphCtu+nKd+mfo8whOTeFkgjMWvk8iCSlqgibKSa8= github.com/scylladb/rclone v1.54.1-0.20240312172628-afe1fd2aa65e/go.mod h1:JGZp4EvCUK+6AM1Fe1dye5xvihTc/Bk0WnHHSCJOePM= +github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241030073626-e409ae491c83 h1:1E64zMOGQ1N3FFIpkAKKF/XVBo6WZEL3tuXHgluqDCk= +github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241030073626-e409ae491c83/go.mod h1:CkiJ7/3IfAZekjx7dFltltVI7HS1e7k8QKsofX6uitM= +github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83 h1:6kyuL5XXRTPbBSUessmfz6TxhYgOiIHCwC63wvDExgQ= +github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83/go.mod h1:+sPCx2oaOXmMpy/ODNNEDGJ7vCghBeKP4S7xEfMI+eA= +github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83 h1:+2wvERSjYAhGushwWpqw8EemmntHOIcCjHCWzEdpXcM= +github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83/go.mod h1:Oxfuz1XcXi9iV4ggSGfQdn+p6gPz6djPOegRMMe/6/s= github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4 h1:8qmTC5ByIXO3GP/IzBkxcZ/99VITvnIETDhdFz/om7A= github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4/go.mod h1:C1a7PQSMz9NShzorzCiG2fk9+xuCgLkPeCvMHYR2OWg= github.com/sirupsen/logrus v1.7.0 h1:ShrD1U9pZB12TX0cVy0DtePoCH97K8EtX+mg7ZARUtM= diff --git a/pkg/schema/table/table.go b/pkg/schema/table/table.go index 5dcbeefcc6..6163163e3c 100644 --- a/pkg/schema/table/table.go +++ b/pkg/schema/table/table.go @@ -221,6 +221,7 @@ var ( "restore_completed_at", "restore_started_at", "run_id", + "shard_cnt", "skipped", "sstable_id", "table_name", diff --git a/pkg/service/restore/model.go b/pkg/service/restore/model.go index 8b9df2424b..2dc654399c 100644 --- a/pkg/service/restore/model.go +++ b/pkg/service/restore/model.go @@ -11,8 +11,6 @@ import ( "github.com/gocql/gocql" "github.com/pkg/errors" "github.com/scylladb/gocqlx/v2" - "github.com/scylladb/gocqlx/v2/qb" - "github.com/scylladb/scylla-manager/v3/pkg/schema/table" "github.com/scylladb/scylla-manager/v3/pkg/scyllaclient" "github.com/scylladb/scylla-manager/v3/pkg/service/repair" @@ -192,6 +190,7 @@ type RunProgress struct { SSTableID []string `db:"sstable_id"` Host string // IP of the node to which SSTables are downloaded. + ShardCnt int64 // Host shard count used for bandwidth per shard calculation. AgentJobID int64 DownloadStartedAt *time.Time @@ -205,34 +204,6 @@ type RunProgress struct { VersionedProgress int64 } -// ForEachTableProgress iterates over all TableProgress belonging to the same run/manifest/table as the receiver. -func (pr *RunProgress) ForEachTableProgress(session gocqlx.Session, cb func(*RunProgress)) error { - q := qb.Select(table.RestoreRunProgress.Name()).Where( - qb.Eq("cluster_id"), - qb.Eq("task_id"), - qb.Eq("run_id"), - qb.Eq("manifest_path"), - qb.Eq("keyspace_name"), - qb.Eq("table_name"), - ).Query(session) - defer q.Release() - - iter := q.BindMap(qb.M{ - "cluster_id": pr.ClusterID, - "task_id": pr.TaskID, - "run_id": pr.RunID, - "manifest_path": pr.RemoteSSTableDir, - "keyspace_name": pr.Keyspace, - "table_name": pr.Table, - }).Iter() - - res := new(RunProgress) - for iter.StructScan(res) { - cb(res) - } - return iter.Close() -} - func (pr *RunProgress) setRestoreStartedAt() { t := timeutc.Now() pr.RestoreStartedAt = &t @@ -263,6 +234,7 @@ type Progress struct { SnapshotTag string `json:"snapshot_tag"` Keyspaces []KeyspaceProgress `json:"keyspaces,omitempty"` + Hosts []HostProgress `json:"hosts,omitempty"` Views []ViewProgress `json:"views,omitempty"` Stage Stage `json:"stage"` } @@ -275,6 +247,16 @@ type KeyspaceProgress struct { Tables []TableProgress `json:"tables,omitempty"` } +// HostProgress groups restore progress for the host. +type HostProgress struct { + Host string `json:"host"` + ShardCnt int64 `json:"shard_cnt"` + DownloadedBytes int64 `json:"downloaded_bytes"` + DownloadDuration int64 `json:"download_duration"` + StreamedBytes int64 `json:"streamed_bytes"` + StreamDuration int64 `json:"stream_duration"` +} + // TableProgress defines restore progress for the table. type TableProgress struct { progress diff --git a/pkg/service/restore/progress.go b/pkg/service/restore/progress.go index f99389d00e..58607b1b27 100644 --- a/pkg/service/restore/progress.go +++ b/pkg/service/restore/progress.go @@ -32,8 +32,9 @@ func (w *worker) aggregateProgress(ctx context.Context) (Progress, error) { SnapshotTag: w.run.SnapshotTag, Stage: w.run.Stage, } - tableMap = make(map[tableKey]*TableProgress) - key tableKey + tableMap = make(map[tableKey]*TableProgress) + key tableKey + hostProgress = make(map[string]HostProgress) ) // Initialize tables and their size @@ -54,7 +55,20 @@ func (w *worker) aggregateProgress(ctx context.Context) (Progress, error) { } // Initialize tables' progress - err := forEachProgress(w.session, w.run.ClusterID, w.run.TaskID, w.run.ID, aggregateRestoreTableProgress(tableMap)) + atp := aggregateRestoreTableProgress(tableMap) + err := forEachProgress(w.session, w.run.ClusterID, w.run.TaskID, w.run.ID, func(runProgress *RunProgress) { + atp(runProgress) + hp := hostProgress[runProgress.Host] + hp.Host = runProgress.Host + hp.ShardCnt = runProgress.ShardCnt + hp.DownloadedBytes += runProgress.Downloaded + hp.DownloadDuration += timeSub(runProgress.DownloadStartedAt, runProgress.DownloadCompletedAt).Milliseconds() + if runProgress.RestoreCompletedAt != nil { + hp.StreamedBytes += runProgress.Downloaded + hp.StreamDuration += timeSub(runProgress.RestoreStartedAt, runProgress.RestoreCompletedAt).Milliseconds() + } + hostProgress[runProgress.Host] = hp + }) if err != nil { return p, errors.Wrap(err, "iterate over restore progress") } @@ -106,6 +120,9 @@ func (w *worker) aggregateProgress(ctx context.Context) (Progress, error) { }) } + for _, hp := range hostProgress { + p.Hosts = append(p.Hosts, hp) + } return p, nil } @@ -216,5 +233,5 @@ func timeSub(start, end *time.Time) time.Duration { } return endV.Sub(*start) } - return 0 + return time.Duration(0) } diff --git a/pkg/service/restore/service.go b/pkg/service/restore/service.go index dfe1fbc7a6..5de6f14a2c 100644 --- a/pkg/service/restore/service.go +++ b/pkg/service/restore/service.go @@ -115,7 +115,7 @@ func (s *Service) Restore(ctx context.Context, clusterID, taskID, runID uuid.UUI for _, unit := range w.run.Units { totalBytesToRestore += unit.Size } - tw, workerErr := newTablesWorker(w, s.repairSvc, totalBytesToRestore) + tw, workerErr := newTablesWorker(ctx, w, s.repairSvc, totalBytesToRestore) if workerErr != nil { err = workerErr } else { diff --git a/pkg/service/restore/tables_worker.go b/pkg/service/restore/tables_worker.go index ee5f33c1ee..91c87c73ff 100644 --- a/pkg/service/restore/tables_worker.go +++ b/pkg/service/restore/tables_worker.go @@ -21,6 +21,8 @@ import ( type tablesWorker struct { worker + hosts []string + hostShardCnt map[string]uint tableVersion map[TableName]string repairSvc *repair.Service progress *TotalRestoreProgress @@ -65,7 +67,7 @@ func (p *TotalRestoreProgress) Update(bytesRestored int64) { p.restoredBytes += bytesRestored } -func newTablesWorker(w worker, repairSvc *repair.Service, totalBytes int64) (*tablesWorker, error) { +func newTablesWorker(ctx context.Context, w worker, repairSvc *repair.Service, totalBytes int64) (*tablesWorker, error) { versions := make(map[TableName]string) for _, u := range w.run.Units { for _, t := range u.Tables { @@ -80,8 +82,24 @@ func newTablesWorker(w worker, repairSvc *repair.Service, totalBytes int64) (*ta } } + hostsS := strset.New() + for _, h := range w.target.locationHosts { + hostsS.Add(h...) + } + hosts := hostsS.List() + + hostToShard, err := w.client.HostsShardCount(ctx, hosts) + if err != nil { + return nil, errors.Wrap(err, "get hosts shard count") + } + for h, sh := range hostToShard { + w.logger.Info(ctx, "Host shard count", "host", h, "shards", sh) + } + return &tablesWorker{ worker: w, + hosts: hosts, + hostShardCnt: hostToShard, tableVersion: versions, repairSvc: repairSvc, progress: NewTotalRestoreProgress(totalBytes), @@ -169,29 +187,15 @@ func (w *tablesWorker) stageRestoreData(ctx context.Context) error { } w.initMetrics(workload) - hostsS := strset.New() - for _, h := range w.target.locationHosts { - hostsS.Add(h...) - } - hosts := hostsS.List() - - hostToShard, err := w.client.HostsShardCount(ctx, hosts) - if err != nil { - return errors.Wrap(err, "get hosts shard count") - } - for h, sh := range hostToShard { - w.logger.Info(ctx, "Host shard count", "host", h, "shards", sh) - } - // This defer is outside of target field check for improved safety. // We always want to enable auto compaction outside the restore. defer func() { - if err := w.setAutoCompaction(context.Background(), hosts, true); err != nil { + if err := w.setAutoCompaction(context.Background(), w.hosts, true); err != nil { w.logger.Error(ctx, "Couldn't enable auto compaction", "error", err) } }() if !w.target.AllowCompaction { - if err := w.setAutoCompaction(ctx, hosts, false); err != nil { + if err := w.setAutoCompaction(ctx, w.hosts, false); err != nil { return errors.Wrapf(err, "disable auto compaction") } } @@ -199,25 +203,25 @@ func (w *tablesWorker) stageRestoreData(ctx context.Context) error { // Same as above. // We always want to pin agent to CPUs outside the restore. defer func() { - if err := w.pinAgentCPU(context.Background(), hosts, true); err != nil { + if err := w.pinAgentCPU(context.Background(), w.hosts, true); err != nil { w.logger.Error(ctx, "Couldn't re-pin agent to CPUs", "error", err) } }() if w.target.UnpinAgentCPU { - if err := w.pinAgentCPU(ctx, hosts, false); err != nil { + if err := w.pinAgentCPU(ctx, w.hosts, false); err != nil { return errors.Wrapf(err, "unpin agent from CPUs") } } - bd := newBatchDispatcher(workload, w.target.BatchSize, hostToShard, w.target.locationHosts) + bd := newBatchDispatcher(workload, w.target.BatchSize, w.hostShardCnt, w.target.locationHosts) f := func(n int) error { - host := hosts[n] + host := w.hosts[n] dc, err := w.client.HostDatacenter(ctx, host) if err != nil { return errors.Wrapf(err, "get host %s data center", host) } - hi := w.hostInfo(host, dc, hostToShard[host]) + hi := w.hostInfo(host, dc, w.hostShardCnt[host]) w.logger.Info(ctx, "Host info", "host", hi.Host, "transfers", hi.Transfers, "rate limit", hi.RateLimit) for { if ctx.Err() != nil { @@ -252,12 +256,12 @@ func (w *tablesWorker) stageRestoreData(ctx context.Context) error { notify := func(n int, err error) { w.logger.Error(ctx, "Failed to restore files on host", - "host", hosts[n], + "host", w.hosts[n], "error", err, ) } - err = parallel.Run(len(hosts), w.target.Parallel, f, notify) + err = parallel.Run(len(w.hosts), w.target.Parallel, f, notify) if err == nil { if ctx.Err() != nil { return ctx.Err() diff --git a/pkg/service/restore/tablesdir_worker.go b/pkg/service/restore/tablesdir_worker.go index 0f3d81d839..eae7c386cd 100644 --- a/pkg/service/restore/tablesdir_worker.go +++ b/pkg/service/restore/tablesdir_worker.go @@ -110,6 +110,7 @@ func (w *tablesWorker) newRunProgress(ctx context.Context, hi HostInfo, b batch) Keyspace: b.Keyspace, Table: b.Table, Host: hi.Host, + ShardCnt: int64(w.hostShardCnt[hi.Host]), AgentJobID: jobID, SSTableID: b.IDs(), VersionedProgress: versionedPr, diff --git a/schema/v3.4.0.cql b/schema/v3.4.0.cql index 4fe13883cd..25e0924c20 100644 --- a/schema/v3.4.0.cql +++ b/schema/v3.4.0.cql @@ -1,4 +1,6 @@ ALTER TABLE restore_run DROP location; ALTER TABLE restore_run DROP manifest_path; ALTER TABLE restore_run DROP keyspace_name; -ALTER TABLE restore_run DROP table_name; \ No newline at end of file +ALTER TABLE restore_run DROP table_name; + +ALTER TABLE restore_run_progress ADD shard_cnt bigint; \ No newline at end of file diff --git a/v3/pkg/managerclient/go.mod b/v3/pkg/managerclient/go.mod index c8d246ac12..f739fbe1bb 100644 --- a/v3/pkg/managerclient/go.mod +++ b/v3/pkg/managerclient/go.mod @@ -10,8 +10,8 @@ require ( github.com/lnquy/cron v1.1.1 github.com/pkg/errors v0.9.1 github.com/scylladb/go-set v1.0.2 - github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241028110806-78e39cceec83 - github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241028110806-78e39cceec83 + github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83 + github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83 github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4 ) diff --git a/v3/pkg/managerclient/go.sum b/v3/pkg/managerclient/go.sum index e0c5dd2952..1ecdbc4ef5 100644 --- a/v3/pkg/managerclient/go.sum +++ b/v3/pkg/managerclient/go.sum @@ -84,10 +84,10 @@ github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDN github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/scylladb/go-set v1.0.2 h1:SkvlMCKhP0wyyct6j+0IHJkBkSZL+TDzZ4E7f7BCcRE= github.com/scylladb/go-set v1.0.2/go.mod h1:DkpGd78rljTxKAnTDPFqXSGxvETQnJyuSOQwsHycqfs= -github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241028110806-78e39cceec83 h1:yy3k0OEYbsmgDenYuJd7B/nftAUI7VMB/WUoX1Iv/6I= -github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241028110806-78e39cceec83/go.mod h1:+sPCx2oaOXmMpy/ODNNEDGJ7vCghBeKP4S7xEfMI+eA= -github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241028110806-78e39cceec83 h1:cCbhUYGzQ/xbjSN+w3g/H2ZcIoEn6OGRu3l5a+et/vs= -github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241028110806-78e39cceec83/go.mod h1:Oxfuz1XcXi9iV4ggSGfQdn+p6gPz6djPOegRMMe/6/s= +github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83 h1:6kyuL5XXRTPbBSUessmfz6TxhYgOiIHCwC63wvDExgQ= +github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83/go.mod h1:+sPCx2oaOXmMpy/ODNNEDGJ7vCghBeKP4S7xEfMI+eA= +github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83 h1:+2wvERSjYAhGushwWpqw8EemmntHOIcCjHCWzEdpXcM= +github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83/go.mod h1:Oxfuz1XcXi9iV4ggSGfQdn+p6gPz6djPOegRMMe/6/s= github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4 h1:8qmTC5ByIXO3GP/IzBkxcZ/99VITvnIETDhdFz/om7A= github.com/scylladb/termtables v0.0.0-20191203121021-c4c0b6d42ff4/go.mod h1:C1a7PQSMz9NShzorzCiG2fk9+xuCgLkPeCvMHYR2OWg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= diff --git a/v3/pkg/managerclient/model.go b/v3/pkg/managerclient/model.go index 68566846ad..bc594eb077 100644 --- a/v3/pkg/managerclient/model.go +++ b/v3/pkg/managerclient/model.go @@ -1021,6 +1021,9 @@ Duration: {{ FormatDuration .StartTime .EndTime }} {{ end -}} {{ with .Progress }}Progress: {{ if ne .Size 0 }}{{ FormatRestoreProgress .Size .Restored .Downloaded .Failed }}{{else}}-{{ end }} Snapshot Tag: {{ .SnapshotTag }} +Bandwidth: + - Download: {{ avgDownload .Hosts }} + - Load&stream: {{ avgStream .Hosts }} {{ else }}Progress: 0% {{ end }} {{- if .Errors -}} @@ -1038,6 +1041,8 @@ func (rp RestoreProgress) addHeader(w io.Writer) error { "FormatError": FormatError, "FormatRestoreProgress": FormatRestoreProgress, "status": rp.status, + "avgDownload": avgDownload, + "avgStream": avgStream, }).Parse(restoreProgressTemplate)) return temp.Execute(w, rp) } @@ -1055,6 +1060,37 @@ func (rp RestoreProgress) status() string { return s } +func avgDownload(hosts []*models.RestoreHostProgress) string { + var bytes, milliseconds, shards int64 + for _, hp := range hosts { + bytes += hp.DownloadedBytes + milliseconds += hp.DownloadDuration + shards += hp.ShardCnt + } + return formatBandwidth(bytes, milliseconds, shards) +} + +func avgStream(hosts []*models.RestoreHostProgress) string { + var bytes, milliseconds, shards int64 + for _, hp := range hosts { + bytes += hp.StreamedBytes + milliseconds += hp.StreamDuration + shards += hp.ShardCnt + } + return formatBandwidth(bytes, milliseconds, shards) +} + +func formatBandwidth(bytes, milliseconds, shards int64) string { + if milliseconds <= 0 { + return "unknown" + } + bs := bytes * 1000 / milliseconds + if shards <= 0 { + return FormatSizeSuffix(bs) + "/s" + } + return FormatSizeSuffix(bs/shards) + "/s/shard" +} + func (rp RestoreProgress) hideKeyspace(keyspace string) bool { if rp.KeyspaceFilter.Size() > 0 { return rp.KeyspaceFilter.FirstMatch(keyspace) == -1 @@ -1082,10 +1118,15 @@ func (rp RestoreProgress) Render(w io.Writer) error { } } - if rp.Detailed && rp.Progress.Size > 0 { - if err := rp.addTableProgress(w); err != nil { + if rp.Detailed { + if err := rp.addHostProgress(w); err != nil { return err } + if rp.Progress.Size > 0 { + if err := rp.addTableProgress(w); err != nil { + return err + } + } } // Check if there is repair progress to display @@ -1211,6 +1252,29 @@ func (rp RestoreProgress) addTableProgress(w io.Writer) error { return nil } +func (rp RestoreProgress) addHostProgress(w io.Writer) error { + _, _ = fmt.Fprintf(w, "\nHosts info\n") + t := table.New("Host", "Shards", "Download bandwidth", "Download duration", "Load&stream bandwidth", "Load&stream duration") + for i, hp := range rp.Progress.Hosts { + if i > 0 { + t.AddSeparator() + } + t.AddRow( + hp.Host, + hp.ShardCnt, + formatBandwidth(hp.DownloadedBytes, hp.DownloadDuration, hp.ShardCnt), + FormatMsDuration(hp.DownloadDuration), + formatBandwidth(hp.StreamedBytes, hp.StreamDuration, hp.ShardCnt), + FormatMsDuration(hp.StreamDuration), + ) + } + t.SetColumnAlignment(termtables.AlignRight, 1, 2, 3, 4, 5) + if _, err := w.Write([]byte(t.String())); err != nil { + return err + } + return nil +} + func (rp RestoreProgress) addViewProgress(w io.Writer) { if len(rp.Progress.Views) == 0 { return diff --git a/v3/pkg/managerclient/sizesuffix.go b/v3/pkg/managerclient/sizesuffix.go index 173e2b6513..49ada81e4a 100644 --- a/v3/pkg/managerclient/sizesuffix.go +++ b/v3/pkg/managerclient/sizesuffix.go @@ -37,25 +37,25 @@ func (x SizeSuffix) string() (string, string) { case x < 0: return "off", "" case x == 0: - return "0", "" + return "0", "B" case x < 1<<10: scaled = float64(x) - suffix = "" + suffix = "B" case x < 1<<20: scaled = float64(x) / (1 << 10) - suffix = "k" + suffix = "KiB" case x < 1<<30: scaled = float64(x) / (1 << 20) - suffix = "M" + suffix = "MiB" case x < 1<<40: scaled = float64(x) / (1 << 30) - suffix = "G" + suffix = "GiB" case x < 1<<50: scaled = float64(x) / (1 << 40) - suffix = "T" + suffix = "TiB" default: scaled = float64(x) / (1 << 50) - suffix = "P" + suffix = "PiB" } if math.Floor(scaled) == scaled { return fmt.Sprintf("%.0f", scaled), suffix diff --git a/v3/pkg/managerclient/utils.go b/v3/pkg/managerclient/utils.go index 4a5fe69f22..850c079d70 100644 --- a/v3/pkg/managerclient/utils.go +++ b/v3/pkg/managerclient/utils.go @@ -191,10 +191,10 @@ func FormatDuration(t0, t1 strfmt.DateTime) string { return d.Truncate(time.Second).String() } -// FormatMsDuration returns string representation of duration as number of -// milliseconds. +// FormatMsDuration returns string representation of given amount of milliseconds +// as duration rounded up to a full second. func FormatMsDuration(d int64) string { - return (time.Duration(d) * time.Millisecond).Truncate(time.Second).String() + return (time.Duration(d)*time.Millisecond + time.Second - 1).Truncate(time.Second).String() } func isZero(t strfmt.DateTime) bool { diff --git a/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/LICENSE b/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/LICENSE new file mode 100644 index 0000000000..a155a625fc --- /dev/null +++ b/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/LICENSE @@ -0,0 +1,62 @@ +Scylla SOURCE AVAILABLE LICENSE (SSAL) AGREEMENT Last Update: June 30, 2021 + +This Agreement sets forth the terms on which the Licensor makes available the Software. BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY OF THE SOFTWARE, YOU AGREE TO THE TERMS AND CONDITIONS OF THIS AGREEMENT. IF YOU DO NOT AGREE TO SUCH TERMS AND CONDITIONS, YOU MUST NOT USE THE SOFTWARE. + +If you are receiving the Software on behalf of a legal entity, you represent and warrant that you have the actual authority to agree to the terms and conditions of this agreement on behalf of such entity. + +The terms below have the meanings set forth below for purposes of this Agreement: + +Agreement: this Scylla Source Available License Agreement. + +Database Product: any of the following products or services: + +(a) Scylla Manager; + +(b) a product or service exposing the Scylla Manager API + +License: the Scylla Source Available License described in Section 1. + +Licensor: as indicated in the source code license. + +Modification: a modification of the Software made by You under the License, Section 1.1(c). + +Scylla: the open source Scylla software as described in scylladb.com + +Software: certain software components designed to work with Scylla NoSQL Database and provided to you under this Agreement. + +You: the recipient of this Software, an individual, or the entity on whose behalf you are receiving the Software. + +Your Application: an application developed by or for You, where such application is not a Database Product. + +LICENSE GRANT AND CONDITIONS +1.1 Subject to the terms and conditions of this Section 1, Licensor hereby grants to You a non-exclusive, royalty-free, worldwide, non-transferable license during the term of this Agreement to: + +a) distribute or make available the Software or your Modifications under the terms of this +Agreement, only as part of Your Application, so long as you include the following notice on any copy you distribute: “This software is subject to the terms of the Scylla Source Available License Agreement”. + +(b) use the Software, or your Modifications, only as part of Your Application, but not in connection + +with any Database Product that is distributed or otherwise made available by any third party. + +(c) modify the Software, provided that Modifications remain subject to the terms of this License. + +(d) reproduce the Software as necessary for the above. + +1.3. Notices. + +On all copies of the Software that you make, you must retain all copyright or other proprietary notices. + +TERM AND TERMINATION. +This Agreement will continue unless and until earlier terminated as set forth herein. If You breach any of its conditions or obligations under this Agreement, this Agreement will terminate automatically and the licenses granted herein will terminate automatically. + +INTELLECTUAL PROPERTY. +As between the parties, Licensor retains all right, title, and interest in the Software, and to Scylla or other Licensor trademarks or service marks, and all intellectual property rights therein. Licensor hereby reserves all rights not expressly granted to You in this Agreement. + +DISCLAIMER. +TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR HEREBY DISCLAIMS ANY AND ALL WARRANTIES AND CONDITIONS, EXPRESS, IMPLIED, STATUTORY, OR OTHERWISE, AND SPECIFICALLY DISCLAIMS ANY WARRANTY OF MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE, WITH RESPECT TO THE SOFTWARE. Licensor has no obligation to support the Software. + +LIMITATION OF LIABILITY. TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR WILL NOT BE LIABLE FOR ANY DAMAGES OF ANY KIND, INCLUDING BUT NOT LIMITED TO, LOST PROFITS OR ANY CONSEQUENTIAL, SPECIAL, INCIDENTAL, INDIRECT, OR DIRECT DAMAGES, ARISING OUT OF OR RELATING TO THIS AGREEMENT. +GENERAL. You are not authorized to assign Your rights under this Agreement to any third party. Licensor may freely assign its rights under this Agreement to any third party. This Agreement is the entire agreement between the parties on the subject matter hereof. No amendment or modification hereof will be valid or binding upon the parties unless made in writing and signed by the duly authorized representatives of both parties. In the event that any provision, including without limitation any condition, of this Agreement is held to be unenforceable, this Agreement and all licenses and rights granted hereunder will immediately terminate. Failure by Licensor to exercise any right hereunder will not be construed as a waiver of any subsequent breach of that right or as a waiver of any other right. This Agreement will be governed by and interpreted in accordance with the laws of the state of California, without reference to its conflict of laws principles. If You are located within the United States, all disputes arising out of this Agreement are subject to the exclusive jurisdiction of courts located in Santa Clara County, California. USA. If You are located outside of the United States, any dispute, controversy or claim arising out of or relating to this Agreement will be referred to and finally determined by arbitration in accordance with the JAMS before a single arbitrator in Santa Clara County, California. Judgment upon the award rendered by the arbitrator may be entered in any court having jurisdiction thereof. + +This license is base on REDIS SOURCE AVAILABLE LICENSE (RSAL) AGREEMENT with some updates +https://redislabs.com/wp-content/uploads/2019/09/redis-source-available-license.pdf diff --git a/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/model.go b/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/model.go index d7d920f1fe..68566846ad 100644 --- a/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/model.go +++ b/vendor/github.com/scylladb/scylla-manager/v3/pkg/managerclient/model.go @@ -462,6 +462,8 @@ Snapshot Tag: {{ .SnapshotTag }} Batch Size: {{ .BatchSize }} Parallel: {{ .Parallel }} Transfers: {{ .Transfers }} +Compaction: {{ if .AllowCompaction -}} allowed {{ else -}} not allowed {{ end }} +Agent CPU: {{ if .UnpinAgentCPU -}} unpinned {{ else -}} pinned {{ end }} Download Rate Limits: {{- if .RateLimit -}} {{ range .RateLimit }} @@ -470,8 +472,6 @@ Download Rate Limits: {{- else }} - Unlimited {{- end }} -Compaction Allowed: {{ .AllowCompaction }} -Agent CPU Unpinned: {{ .UnpinAgentCPU }} ` // Render implements Renderer interface. @@ -1091,19 +1091,7 @@ func (rp RestoreProgress) Render(w io.Writer) error { // Check if there is repair progress to display if rp.Progress.RepairProgress != nil { fmt.Fprintf(w, "\nPost-restore repair progress\n") - - repairRunPr := &models.TaskRunRepairProgress{ - Progress: rp.Progress.RepairProgress, - Run: rp.Run, - } - repairPr := RepairProgress{ - TaskRunRepairProgress: repairRunPr, - Task: rp.Task, - Detailed: rp.Detailed, - keyspaceFilter: rp.KeyspaceFilter, - } - - if err := repairPr.Render(w); err != nil { + if err := rp.postRestoreRepairProgress().Render(w); err != nil { return err } } @@ -1112,6 +1100,52 @@ func (rp RestoreProgress) Render(w io.Writer) error { return nil } +func (rp RestoreProgress) postRestoreRepairProgress() RepairProgress { + repair := rp.Progress.RepairProgress + + repairRun := &models.TaskRun{ + ClusterID: rp.Task.ClusterID, + Type: RepairTask, + } + if repair.StartedAt != nil { + repairRun.StartTime = *repair.StartedAt + } + if repair.CompletedAt != nil { + repairRun.EndTime = *repair.CompletedAt + } + if rp.Progress.Stage == RestoreStageRepair { + repairRun.Cause = rp.Run.Cause + repairRun.Status = rp.Run.Status + } else { + switch { + case repair.Success == repair.TokenRanges: + repairRun.Status = TaskStatusDone + case repair.Error > 0: + repairRun.Status = TaskStatusError + } + } + + repairTask := &models.Task{ + ClusterID: rp.Task.ClusterID, + Enabled: true, + Properties: map[string]any{ + "intensity": repair.Intensity, + "parallel": repair.Parallel, + }, + Type: RepairTask, + } + + return RepairProgress{ + TaskRunRepairProgress: &models.TaskRunRepairProgress{ + Progress: repair, + Run: repairRun, + }, + Task: repairTask, + Detailed: rp.Detailed, + keyspaceFilter: rp.KeyspaceFilter, + } +} + func (rp RestoreProgress) addKeyspaceProgress(t *table.Table) { t.AddRow("Keyspace", "Progress", "Size", "Success", "Downloaded", "Failed") t.AddSeparator() diff --git a/vendor/github.com/scylladb/scylla-manager/v3/pkg/util/LICENSE b/vendor/github.com/scylladb/scylla-manager/v3/pkg/util/LICENSE new file mode 100644 index 0000000000..a155a625fc --- /dev/null +++ b/vendor/github.com/scylladb/scylla-manager/v3/pkg/util/LICENSE @@ -0,0 +1,62 @@ +Scylla SOURCE AVAILABLE LICENSE (SSAL) AGREEMENT Last Update: June 30, 2021 + +This Agreement sets forth the terms on which the Licensor makes available the Software. BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY OF THE SOFTWARE, YOU AGREE TO THE TERMS AND CONDITIONS OF THIS AGREEMENT. IF YOU DO NOT AGREE TO SUCH TERMS AND CONDITIONS, YOU MUST NOT USE THE SOFTWARE. + +If you are receiving the Software on behalf of a legal entity, you represent and warrant that you have the actual authority to agree to the terms and conditions of this agreement on behalf of such entity. + +The terms below have the meanings set forth below for purposes of this Agreement: + +Agreement: this Scylla Source Available License Agreement. + +Database Product: any of the following products or services: + +(a) Scylla Manager; + +(b) a product or service exposing the Scylla Manager API + +License: the Scylla Source Available License described in Section 1. + +Licensor: as indicated in the source code license. + +Modification: a modification of the Software made by You under the License, Section 1.1(c). + +Scylla: the open source Scylla software as described in scylladb.com + +Software: certain software components designed to work with Scylla NoSQL Database and provided to you under this Agreement. + +You: the recipient of this Software, an individual, or the entity on whose behalf you are receiving the Software. + +Your Application: an application developed by or for You, where such application is not a Database Product. + +LICENSE GRANT AND CONDITIONS +1.1 Subject to the terms and conditions of this Section 1, Licensor hereby grants to You a non-exclusive, royalty-free, worldwide, non-transferable license during the term of this Agreement to: + +a) distribute or make available the Software or your Modifications under the terms of this +Agreement, only as part of Your Application, so long as you include the following notice on any copy you distribute: “This software is subject to the terms of the Scylla Source Available License Agreement”. + +(b) use the Software, or your Modifications, only as part of Your Application, but not in connection + +with any Database Product that is distributed or otherwise made available by any third party. + +(c) modify the Software, provided that Modifications remain subject to the terms of this License. + +(d) reproduce the Software as necessary for the above. + +1.3. Notices. + +On all copies of the Software that you make, you must retain all copyright or other proprietary notices. + +TERM AND TERMINATION. +This Agreement will continue unless and until earlier terminated as set forth herein. If You breach any of its conditions or obligations under this Agreement, this Agreement will terminate automatically and the licenses granted herein will terminate automatically. + +INTELLECTUAL PROPERTY. +As between the parties, Licensor retains all right, title, and interest in the Software, and to Scylla or other Licensor trademarks or service marks, and all intellectual property rights therein. Licensor hereby reserves all rights not expressly granted to You in this Agreement. + +DISCLAIMER. +TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR HEREBY DISCLAIMS ANY AND ALL WARRANTIES AND CONDITIONS, EXPRESS, IMPLIED, STATUTORY, OR OTHERWISE, AND SPECIFICALLY DISCLAIMS ANY WARRANTY OF MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE, WITH RESPECT TO THE SOFTWARE. Licensor has no obligation to support the Software. + +LIMITATION OF LIABILITY. TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR WILL NOT BE LIABLE FOR ANY DAMAGES OF ANY KIND, INCLUDING BUT NOT LIMITED TO, LOST PROFITS OR ANY CONSEQUENTIAL, SPECIAL, INCIDENTAL, INDIRECT, OR DIRECT DAMAGES, ARISING OUT OF OR RELATING TO THIS AGREEMENT. +GENERAL. You are not authorized to assign Your rights under this Agreement to any third party. Licensor may freely assign its rights under this Agreement to any third party. This Agreement is the entire agreement between the parties on the subject matter hereof. No amendment or modification hereof will be valid or binding upon the parties unless made in writing and signed by the duly authorized representatives of both parties. In the event that any provision, including without limitation any condition, of this Agreement is held to be unenforceable, this Agreement and all licenses and rights granted hereunder will immediately terminate. Failure by Licensor to exercise any right hereunder will not be construed as a waiver of any subsequent breach of that right or as a waiver of any other right. This Agreement will be governed by and interpreted in accordance with the laws of the state of California, without reference to its conflict of laws principles. If You are located within the United States, all disputes arising out of this Agreement are subject to the exclusive jurisdiction of courts located in Santa Clara County, California. USA. If You are located outside of the United States, any dispute, controversy or claim arising out of or relating to this Agreement will be referred to and finally determined by arbitration in accordance with the JAMS before a single arbitrator in Santa Clara County, California. Judgment upon the award rendered by the arbitrator may be entered in any court having jurisdiction thereof. + +This license is base on REDIS SOURCE AVAILABLE LICENSE (RSAL) AGREEMENT with some updates +https://redislabs.com/wp-content/uploads/2019/09/redis-source-available-license.pdf diff --git a/vendor/github.com/scylladb/scylla-manager/v3/swagger/LICENSE b/vendor/github.com/scylladb/scylla-manager/v3/swagger/LICENSE new file mode 100644 index 0000000000..a155a625fc --- /dev/null +++ b/vendor/github.com/scylladb/scylla-manager/v3/swagger/LICENSE @@ -0,0 +1,62 @@ +Scylla SOURCE AVAILABLE LICENSE (SSAL) AGREEMENT Last Update: June 30, 2021 + +This Agreement sets forth the terms on which the Licensor makes available the Software. BY INSTALLING, DOWNLOADING, ACCESSING, USING OR DISTRIBUTING ANY OF THE SOFTWARE, YOU AGREE TO THE TERMS AND CONDITIONS OF THIS AGREEMENT. IF YOU DO NOT AGREE TO SUCH TERMS AND CONDITIONS, YOU MUST NOT USE THE SOFTWARE. + +If you are receiving the Software on behalf of a legal entity, you represent and warrant that you have the actual authority to agree to the terms and conditions of this agreement on behalf of such entity. + +The terms below have the meanings set forth below for purposes of this Agreement: + +Agreement: this Scylla Source Available License Agreement. + +Database Product: any of the following products or services: + +(a) Scylla Manager; + +(b) a product or service exposing the Scylla Manager API + +License: the Scylla Source Available License described in Section 1. + +Licensor: as indicated in the source code license. + +Modification: a modification of the Software made by You under the License, Section 1.1(c). + +Scylla: the open source Scylla software as described in scylladb.com + +Software: certain software components designed to work with Scylla NoSQL Database and provided to you under this Agreement. + +You: the recipient of this Software, an individual, or the entity on whose behalf you are receiving the Software. + +Your Application: an application developed by or for You, where such application is not a Database Product. + +LICENSE GRANT AND CONDITIONS +1.1 Subject to the terms and conditions of this Section 1, Licensor hereby grants to You a non-exclusive, royalty-free, worldwide, non-transferable license during the term of this Agreement to: + +a) distribute or make available the Software or your Modifications under the terms of this +Agreement, only as part of Your Application, so long as you include the following notice on any copy you distribute: “This software is subject to the terms of the Scylla Source Available License Agreement”. + +(b) use the Software, or your Modifications, only as part of Your Application, but not in connection + +with any Database Product that is distributed or otherwise made available by any third party. + +(c) modify the Software, provided that Modifications remain subject to the terms of this License. + +(d) reproduce the Software as necessary for the above. + +1.3. Notices. + +On all copies of the Software that you make, you must retain all copyright or other proprietary notices. + +TERM AND TERMINATION. +This Agreement will continue unless and until earlier terminated as set forth herein. If You breach any of its conditions or obligations under this Agreement, this Agreement will terminate automatically and the licenses granted herein will terminate automatically. + +INTELLECTUAL PROPERTY. +As between the parties, Licensor retains all right, title, and interest in the Software, and to Scylla or other Licensor trademarks or service marks, and all intellectual property rights therein. Licensor hereby reserves all rights not expressly granted to You in this Agreement. + +DISCLAIMER. +TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR HEREBY DISCLAIMS ANY AND ALL WARRANTIES AND CONDITIONS, EXPRESS, IMPLIED, STATUTORY, OR OTHERWISE, AND SPECIFICALLY DISCLAIMS ANY WARRANTY OF MERCHANTABILITY OR FITNESS FOR A PARTICULAR PURPOSE, WITH RESPECT TO THE SOFTWARE. Licensor has no obligation to support the Software. + +LIMITATION OF LIABILITY. TO THE EXTENT ALLOWABLE UNDER LAW, LICENSOR WILL NOT BE LIABLE FOR ANY DAMAGES OF ANY KIND, INCLUDING BUT NOT LIMITED TO, LOST PROFITS OR ANY CONSEQUENTIAL, SPECIAL, INCIDENTAL, INDIRECT, OR DIRECT DAMAGES, ARISING OUT OF OR RELATING TO THIS AGREEMENT. +GENERAL. You are not authorized to assign Your rights under this Agreement to any third party. Licensor may freely assign its rights under this Agreement to any third party. This Agreement is the entire agreement between the parties on the subject matter hereof. No amendment or modification hereof will be valid or binding upon the parties unless made in writing and signed by the duly authorized representatives of both parties. In the event that any provision, including without limitation any condition, of this Agreement is held to be unenforceable, this Agreement and all licenses and rights granted hereunder will immediately terminate. Failure by Licensor to exercise any right hereunder will not be construed as a waiver of any subsequent breach of that right or as a waiver of any other right. This Agreement will be governed by and interpreted in accordance with the laws of the state of California, without reference to its conflict of laws principles. If You are located within the United States, all disputes arising out of this Agreement are subject to the exclusive jurisdiction of courts located in Santa Clara County, California. USA. If You are located outside of the United States, any dispute, controversy or claim arising out of or relating to this Agreement will be referred to and finally determined by arbitration in accordance with the JAMS before a single arbitrator in Santa Clara County, California. Judgment upon the award rendered by the arbitrator may be entered in any court having jurisdiction thereof. + +This license is base on REDIS SOURCE AVAILABLE LICENSE (RSAL) AGREEMENT with some updates +https://redislabs.com/wp-content/uploads/2019/09/redis-source-available-license.pdf diff --git a/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_host_progress.go b/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_host_progress.go new file mode 100644 index 0000000000..c97ba47aa8 --- /dev/null +++ b/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_host_progress.go @@ -0,0 +1,58 @@ +// Code generated by go-swagger; DO NOT EDIT. + +package models + +// This file was generated by the swagger tool. +// Editing this file might prove futile when you re-run the swagger generate command + +import ( + "github.com/go-openapi/strfmt" + "github.com/go-openapi/swag" +) + +// RestoreHostProgress restore host progress +// +// swagger:model RestoreHostProgress +type RestoreHostProgress struct { + + // Total time spent by host on download in milliseconds + DownloadDuration int64 `json:"download_duration,omitempty"` + + // Total bytes downloaded by host + DownloadedBytes int64 `json:"downloaded_bytes,omitempty"` + + // host + Host string `json:"host,omitempty"` + + // Host shard count + ShardCnt int64 `json:"shard_cnt,omitempty"` + + // Total time spent by host on load&stream in milliseconds + StreamDuration int64 `json:"stream_duration,omitempty"` + + // Total bytes load&streamed by host + StreamedBytes int64 `json:"streamed_bytes,omitempty"` +} + +// Validate validates this restore host progress +func (m *RestoreHostProgress) Validate(formats strfmt.Registry) error { + return nil +} + +// MarshalBinary interface implementation +func (m *RestoreHostProgress) MarshalBinary() ([]byte, error) { + if m == nil { + return nil, nil + } + return swag.WriteJSON(m) +} + +// UnmarshalBinary interface implementation +func (m *RestoreHostProgress) UnmarshalBinary(b []byte) error { + var res RestoreHostProgress + if err := swag.ReadJSON(b, &res); err != nil { + return err + } + *m = res + return nil +} diff --git a/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_progress.go b/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_progress.go index f250ac14ba..e7c63828cb 100644 --- a/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_progress.go +++ b/vendor/github.com/scylladb/scylla-manager/v3/swagger/gen/scylla-manager/models/restore_progress.go @@ -29,6 +29,9 @@ type RestoreProgress struct { // failed Failed int64 `json:"failed,omitempty"` + // hosts + Hosts []*RestoreHostProgress `json:"hosts"` + // keyspaces Keyspaces []*RestoreKeyspaceProgress `json:"keyspaces"` @@ -63,6 +66,10 @@ func (m *RestoreProgress) Validate(formats strfmt.Registry) error { res = append(res, err) } + if err := m.validateHosts(formats); err != nil { + res = append(res, err) + } + if err := m.validateKeyspaces(formats); err != nil { res = append(res, err) } @@ -98,6 +105,31 @@ func (m *RestoreProgress) validateCompletedAt(formats strfmt.Registry) error { return nil } +func (m *RestoreProgress) validateHosts(formats strfmt.Registry) error { + + if swag.IsZero(m.Hosts) { // not required + return nil + } + + for i := 0; i < len(m.Hosts); i++ { + if swag.IsZero(m.Hosts[i]) { // not required + continue + } + + if m.Hosts[i] != nil { + if err := m.Hosts[i].Validate(formats); err != nil { + if ve, ok := err.(*errors.Validation); ok { + return ve.ValidateName("hosts" + "." + strconv.Itoa(i)) + } + return err + } + } + + } + + return nil +} + func (m *RestoreProgress) validateKeyspaces(formats strfmt.Registry) error { if swag.IsZero(m.Keyspaces) { // not required diff --git a/vendor/modules.txt b/vendor/modules.txt index 30af046c4d..88b5b3f2b3 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -395,11 +395,11 @@ github.com/scylladb/gocqlx/v2/dbutil github.com/scylladb/gocqlx/v2/migrate github.com/scylladb/gocqlx/v2/qb github.com/scylladb/gocqlx/v2/table -# github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241028110806-78e39cceec83 => ./v3/pkg/managerclient +# github.com/scylladb/scylla-manager/v3/pkg/managerclient v0.0.0-20241030073626-e409ae491c83 ## explicit; go 1.21.1 github.com/scylladb/scylla-manager/v3/pkg/managerclient github.com/scylladb/scylla-manager/v3/pkg/managerclient/table -# github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241028110806-78e39cceec83 => ./v3/pkg/util +# github.com/scylladb/scylla-manager/v3/pkg/util v0.0.0-20241030073626-e409ae491c83 ## explicit; go 1.21.1 github.com/scylladb/scylla-manager/v3/pkg/util github.com/scylladb/scylla-manager/v3/pkg/util/certutil @@ -431,7 +431,7 @@ github.com/scylladb/scylla-manager/v3/pkg/util/timeutc github.com/scylladb/scylla-manager/v3/pkg/util/uuid github.com/scylladb/scylla-manager/v3/pkg/util/version github.com/scylladb/scylla-manager/v3/pkg/util/workerpool -# github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241028110806-78e39cceec83 => ./v3/swagger +# github.com/scylladb/scylla-manager/v3/swagger v0.0.0-20241030073626-e409ae491c83 ## explicit; go 1.21.1 github.com/scylladb/scylla-manager/v3/swagger/gen/agent/client github.com/scylladb/scylla-manager/v3/swagger/gen/agent/client/operations @@ -758,6 +758,3 @@ gopkg.in/yaml.v2 gopkg.in/yaml.v3 # github.com/gocql/gocql => github.com/scylladb/gocql v1.12.0 # github.com/rclone/rclone => github.com/scylladb/rclone v1.54.1-0.20240312172628-afe1fd2aa65e -# github.com/scylladb/scylla-manager/v3/pkg/managerclient => ./v3/pkg/managerclient -# github.com/scylladb/scylla-manager/v3/pkg/util => ./v3/pkg/util -# github.com/scylladb/scylla-manager/v3/swagger => ./v3/swagger