feature: add new compactor based revision count

What would you like to be added?

Add new compactor based revision count, instead of fixed interval time.

In order to make it happen, the mvcc store needs to export
`CompactNotify` function to notify the compactor that configured number of
write transactions have occured since previsious compaction. The
new compactor can get the revision change and delete out-of-date data in time,
instead of waiting with fixed interval time. The underly bbolt db can
reuse the free pages as soon as possible.

Why is this needed?

In the kubernetes cluster, for instance, argo workflow, there will be batch
requests to create pods , and then there are also a lot of pod status's PATCH
requests, especially when the pod has more than 3 containers. If the burst
requests increase the db size in short time, it will be easy to exceed the max
quota size. And then the cluster admin get involved to defrag, which may casue
long downtime. So, we hope the ETCD can delete the out-of-date data as
soon as possible and slow down the grow of total db size.

Currently, both revision and periodic are based on time. It's not easy
to use fixed interval time to face the unexpected burst update requests.
The new compactor based on revision count can make the admin life easier.
For instance, let's say that average of object size is 50 KiB. The new
compactor will compact based on 10,000 revisions. It's like that ETCD can
compact after new 500 MiB data in, no matter how long ETCD takes to get
new 10,000 revisions. It can handle the burst update requests well.

There are some test results:

* Fixed value size: 10 KiB, Update Rate: 100/s, Total key space: 3,000

```
enchmark put --rate=100 --total=300000 --compact-interval=0 \
  --key-space-size=3000 --key-size=256 --val-size=10240
```

|                      Compactor | DB Total Size | DB InUse Size |
|                             -- | --            |            -- |
| Revision(5min,retension:10000) | 570 MiB       |       208 MiB |
|                   Periodic(1m) | 232 MiB       |       165 MiB |
|                  Periodic(30s) | 151 MiB       |       127 MiB |
|   NewRevision(retension:10000) | 195 MiB       |       187 MiB |

* Random value size: [9 KiB, 11 KiB], Update Rate: 150/s, Total key space: 3,000

```
bnchmark put --rate=150 --total=300000 --compact-interval=0 \
  --key-space-size=3000 --key-size=256 --val-size=10240 \
  --delta-val-size=1024
```

|                      Compactor | DB Total Size | DB InUse Size |
|                             -- | --            |            -- |
| Revision(5min,retension:10000) | 718 MiB       |       554 MiB |
|                   Periodic(1m) | 297 MiB       |       246 MiB |
|                  Periodic(30s) | 185 MiB       |       146 MiB |
|   NewRevision(retension:10000) | 186 MiB       |       178 MiB |

* Random value size: [6 KiB, 14 KiB], Update Rate: 200/s, Total key space: 3,000

```
bnchmark put --rate=200 --total=300000 --compact-interval=0 \
  --key-space-size=3000 --key-size=256 --val-size=10240 \
  --delta-val-size=4096
```

|                      Compactor | DB Total Size | DB InUse Size |
|                             -- | --            |            -- |
| Revision(5min,retension:10000) | 874 MiB       |       221 MiB |
|                   Periodic(1m) | 357 MiB       |       260 MiB |
|                  Periodic(30s) | 215 MiB       |       151 MiB |
|   NewRevision(retension:10000) | 182 MiB       |       176 MiB |

For the burst requests, we needs to use short periodic interval.
Otherwise, the total size will be large. I think the new compactor can
handle it well.

Additional Change:

Currently, the quota system only checks DB total size. However, there
could be a lot of free pages which can be reused to upcoming requests.
Based on this proposal, I also want to extend current quota system with DB's
InUse size.

If the InUse size is less than max quota size, we should allow requests to
update. Since the bbolt might be resized if there is no available
continuous pages, we should setup a hard limit for the overflow, like 1
GiB.

```diff
 // Quota represents an arbitrary quota against arbitrary requests. Each request
@@ -130,7 +134,17 @@ func (b *BackendQuota) Available(v interface{}) bool {
                return true
        }
        // TODO: maybe optimize Backend.Size()
-       return b.be.Size()+int64(cost) < b.maxBackendBytes
+
+       // Since the compact comes with allocatable pages, we should check the
+       // SizeInUse first. If there is no continuous pages for key/value and
+       // the boltdb continues to resize, it should not increase more than 1
+       // GiB. It's hard limitation.
+       //
+       // TODO: It should be enabled by flag.
+       if b.be.Size()+int64(cost)-b.maxBackendBytes >= maxAllowedOverflowBytes(b.maxBackendBytes) {
+               return false
+       }
+       return b.be.SizeInUse()+int64(cost) < b.maxBackendBytes
 }
```

And it's likely to disable NOSPACE alarm if the compact can get much
more free pages. It can reduce downtime.

Signed-off-by: Wei Fu <fuweid89@gmail.com>
This commit is contained in:
Wei Fu 2023-08-14 10:08:23 +00:00
parent 21c4061d5c
commit 4db8df677c
11 changed files with 228 additions and 16 deletions

View File

@ -139,6 +139,14 @@ var (
// revision 5000 when the current revision is 6000. // revision 5000 when the current revision is 6000.
// This runs every 5-minute if enough of logs have proceeded. // This runs every 5-minute if enough of logs have proceeded.
CompactorModeRevision = v3compactor.ModeRevision CompactorModeRevision = v3compactor.ModeRevision
// CompactorModeRevisionThreshold is revision-change-based compaction
// mode for "Config.AutoCompactionMode" field.
// If "AutoCompactionMode" is CompactorModeRevisionThreshold and
// "AutoCompactionRetention" is "10000", it compacts log on revision
// 10000 when the current revision is 20000. This runs if there are
// more than 10000 revisions have occurred since previous compaction.
CompactorModeRevisionThreshold = v3compactor.ModeRevisionThreshold
) )
func init() { func init() {
@ -775,7 +783,7 @@ func (cfg *Config) Validate() error {
} }
switch cfg.AutoCompactionMode { switch cfg.AutoCompactionMode {
case CompactorModeRevision, CompactorModePeriodic: case CompactorModeRevision, CompactorModePeriodic, CompactorModeRevisionThreshold:
case "": case "":
return errors.New("undefined auto-compaction-mode") return errors.New("undefined auto-compaction-mode")
default: default:

View File

@ -888,6 +888,8 @@ func parseCompactionRetention(mode, retention string) (ret time.Duration, err er
ret = time.Duration(int64(h)) ret = time.Duration(int64(h))
case CompactorModePeriodic: case CompactorModePeriodic:
ret = time.Duration(int64(h)) * time.Hour ret = time.Duration(int64(h)) * time.Hour
case CompactorModeRevisionThreshold:
ret = time.Duration(int64(h))
case "": case "":
return 0, errors.New("--auto-compaction-mode is undefined") return 0, errors.New("--auto-compaction-mode is undefined")
} }

View File

@ -28,6 +28,7 @@ import (
const ( const (
ModePeriodic = "periodic" ModePeriodic = "periodic"
ModeRevision = "revision" ModeRevision = "revision"
ModeRevisionThreshold = "revision-threshold"
) )
// Compactor purges old log from the storage periodically. // Compactor purges old log from the storage periodically.
@ -47,6 +48,11 @@ type Compactable interface {
Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error) Compact(ctx context.Context, r *pb.CompactionRequest) (*pb.CompactionResponse, error)
} }
type KvGetter interface {
RevGetter
CompactNotify() chan struct{}
}
type RevGetter interface { type RevGetter interface {
Rev() int64 Rev() int64
} }
@ -56,17 +62,20 @@ func New(
lg *zap.Logger, lg *zap.Logger,
mode string, mode string,
retention time.Duration, retention time.Duration,
rg RevGetter, kg KvGetter,
c Compactable, c Compactable,
) (Compactor, error) { ) (Compactor, error) {
if lg == nil { if lg == nil {
lg = zap.NewNop() lg = zap.NewNop()
} }
switch mode { switch mode {
case ModePeriodic: case ModePeriodic:
return newPeriodic(lg, clockwork.NewRealClock(), retention, rg, c), nil return newPeriodic(lg, clockwork.NewRealClock(), retention, kg, c), nil
case ModeRevision: case ModeRevision:
return newRevision(lg, clockwork.NewRealClock(), int64(retention), rg, c), nil return newRevision(lg, clockwork.NewRealClock(), int64(retention), kg, c), nil
case ModeRevisionThreshold:
return newRevisionThreshold(lg, clockwork.NewRealClock(), kg, c), nil
default: default:
return nil, fmt.Errorf("unsupported compaction mode %s", mode) return nil, fmt.Errorf("unsupported compaction mode %s", mode)
} }

View File

@ -0,0 +1,112 @@
// Copyright 2023 The etcd Authors
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package v3compactor
import (
"context"
"sync"
"time"
"github.com/jonboulle/clockwork"
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
"go.etcd.io/etcd/server/v3/storage/mvcc"
"go.uber.org/zap"
)
type RevisionThreshold struct {
lg *zap.Logger
clock clockwork.Clock
kv KvGetter
c Compactable
ctx context.Context
cancel context.CancelFunc
mu sync.Mutex
paused bool
}
func newRevisionThreshold(lg *zap.Logger, clock clockwork.Clock, kv KvGetter, c Compactable) *RevisionThreshold {
rc := &RevisionThreshold{
lg: lg,
clock: clock,
kv: kv,
c: c,
}
rc.ctx, rc.cancel = context.WithCancel(context.Background())
return rc
}
func (rc *RevisionThreshold) Run() {
go func() {
for {
select {
case <-rc.ctx.Done():
return
case <-rc.kv.CompactNotify():
rc.mu.Lock()
p := rc.paused
rc.mu.Unlock()
if p {
continue
}
}
rev := rc.kv.Rev()
now := time.Now()
rc.lg.Info(
"starting auto revision compaction",
zap.Int64("revision", rev),
)
_, err := rc.c.Compact(rc.ctx, &pb.CompactionRequest{Revision: rev})
if err == nil || err == mvcc.ErrCompacted {
rc.lg.Info(
"completed auto revision compaction",
zap.Int64("revision", rev),
zap.Duration("took", time.Since(now)),
)
} else {
rc.lg.Warn(
"failed auto revision compaction",
zap.Int64("revision", rev),
zap.Error(err),
)
}
}
}()
}
// Stop stops revision-based compactor.
func (rc *RevisionThreshold) Stop() {
rc.cancel()
}
// Pause pauses revision-based compactor.
func (rc *RevisionThreshold) Pause() {
rc.mu.Lock()
rc.paused = true
rc.mu.Unlock()
}
// Resume resumes revision-based compactor.
func (rc *RevisionThreshold) Resume() {
rc.mu.Lock()
rc.paused = false
rc.mu.Unlock()
}

View File

@ -373,6 +373,10 @@ func NewServer(cfg config.ServerConfig) (srv *EtcdServer, err error) {
CompactionBatchLimit: cfg.CompactionBatchLimit, CompactionBatchLimit: cfg.CompactionBatchLimit,
CompactionSleepInterval: cfg.CompactionSleepInterval, CompactionSleepInterval: cfg.CompactionSleepInterval,
} }
if cfg.AutoCompactionMode == v3compactor.ModeRevisionThreshold {
mvccStoreConfig.CompactionNotifyThreshold = int64(cfg.AutoCompactionRetention)
}
srv.kv = mvcc.New(srv.Logger(), srv.be, srv.lessor, mvccStoreConfig) srv.kv = mvcc.New(srv.Logger(), srv.be, srv.lessor, mvccStoreConfig)
srv.corruptionChecker = newCorruptionChecker(cfg.Logger, srv, srv.kv.HashStorage()) srv.corruptionChecker = newCorruptionChecker(cfg.Logger, srv, srv.kv.HashStorage())
@ -387,7 +391,7 @@ func NewServer(cfg config.ServerConfig) (srv *EtcdServer, err error) {
} }
}() }()
if num := cfg.AutoCompactionRetention; num != 0 { if num := cfg.AutoCompactionRetention; num != 0 {
srv.compactor, err = v3compactor.New(cfg.Logger, cfg.AutoCompactionMode, num, srv.kv, srv) srv.compactor, err = v3compactor.New(cfg.Logger, cfg.AutoCompactionMode, num, any(srv.kv).(kvGetter), srv)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -2479,3 +2483,8 @@ func (s *EtcdServer) getTxPostLockInsideApplyHook() func() {
func (s *EtcdServer) CorruptionChecker() CorruptionChecker { func (s *EtcdServer) CorruptionChecker() CorruptionChecker {
return s.corruptionChecker return s.corruptionChecker
} }
type kvGetter interface {
CompactNotify() chan struct{}
Rev() int64
}

View File

@ -53,6 +53,10 @@ var minimumBatchInterval = 10 * time.Millisecond
type StoreConfig struct { type StoreConfig struct {
CompactionBatchLimit int CompactionBatchLimit int
CompactionSleepInterval time.Duration CompactionSleepInterval time.Duration
// CompactionNotifyThreshold is used to guarantee that a notification
// is sent only after configured number of write transactions have
// occured since previsious compaction.
CompactionNotifyThreshold int64
} }
type store struct { type store struct {
@ -77,6 +81,9 @@ type store struct {
currentRev int64 currentRev int64
// compactMainRev is the main revision of the last compaction. // compactMainRev is the main revision of the last compaction.
compactMainRev int64 compactMainRev int64
// compactNotifyCh is used to notify the compactor that it's time to
// compact.
compactNotifyCh chan struct{}
fifoSched schedule.Scheduler fifoSched schedule.Scheduler
@ -107,6 +114,7 @@ func NewStore(lg *zap.Logger, b backend.Backend, le lease.Lessor, cfg StoreConfi
currentRev: 1, currentRev: 1,
compactMainRev: -1, compactMainRev: -1,
compactNotifyCh: make(chan struct{}, 1),
fifoSched: schedule.NewFIFOScheduler(lg), fifoSched: schedule.NewFIFOScheduler(lg),
@ -488,6 +496,9 @@ func restoreChunk(lg *zap.Logger, kvc chan<- revKeyValue, keys, vals [][]byte, k
func (s *store) Close() error { func (s *store) Close() error {
close(s.stopc) close(s.stopc)
if s.compactNotifyCh != nil {
close(s.compactNotifyCh)
}
s.fifoSched.Stop() s.fifoSched.Stop()
return nil return nil
} }
@ -539,3 +550,22 @@ func isTombstone(b []byte) bool {
func (s *store) HashStorage() HashStorage { func (s *store) HashStorage() HashStorage {
return s.hashes return s.hashes
} }
func (s *store) CompactNotify() chan struct{} {
return s.compactNotifyCh
}
func (s *store) doCompactNotify() {
threshold := s.cfg.CompactionNotifyThreshold
if threshold <= 0 {
return
}
if s.currentRev-s.compactMainRev > threshold {
select {
case s.compactNotifyCh <- struct{}{}:
default:
}
}
}

View File

@ -180,6 +180,8 @@ func (tw *storeTxnWrite) End() {
// hold revMu lock to prevent new read txns from opening until writeback. // hold revMu lock to prevent new read txns from opening until writeback.
tw.s.revMu.Lock() tw.s.revMu.Lock()
tw.s.currentRev++ tw.s.currentRev++
tw.s.doCompactNotify()
} }
tw.tx.Unlock() tw.tx.Unlock()
if len(tw.changes) != 0 { if len(tw.changes) != 0 {

View File

@ -31,6 +31,10 @@ const (
// MaxQuotaBytes is the maximum number of bytes suggested for a backend // MaxQuotaBytes is the maximum number of bytes suggested for a backend
// quota. A larger quota may lead to degraded performance. // quota. A larger quota may lead to degraded performance.
MaxQuotaBytes = int64(8 * 1024 * 1024 * 1024) // 8GB MaxQuotaBytes = int64(8 * 1024 * 1024 * 1024) // 8GB
// MaxAllowedOverflowQuotaBytes is the number of bytes the backend size
// can be overflow after exceeding the space quota.
MaxAllowedOverflowQuotaBytes = int64(1024 * 1024 * 1024) // 1GB
) )
// Quota represents an arbitrary quota against arbitrary requests. Each request // Quota represents an arbitrary quota against arbitrary requests. Each request
@ -130,7 +134,17 @@ func (b *BackendQuota) Available(v interface{}) bool {
return true return true
} }
// TODO: maybe optimize Backend.Size() // TODO: maybe optimize Backend.Size()
return b.be.Size()+int64(cost) < b.maxBackendBytes
// Since the compact comes with allocatable pages, we should check the
// SizeInUse first. If there is no continuous pages for key/value and
// the boltdb continues to resize, it should not increase more than 1
// GiB. It's hard limitation.
//
// TODO: It should be enabled by flag.
if b.be.Size()+int64(cost)-b.maxBackendBytes >= maxAllowedOverflowBytes(b.maxBackendBytes) {
return false
}
return b.be.SizeInUse()+int64(cost) < b.maxBackendBytes
} }
func (b *BackendQuota) Cost(v interface{}) int { func (b *BackendQuota) Cost(v interface{}) int {
@ -174,3 +188,11 @@ func costTxn(r *pb.TxnRequest) int {
func (b *BackendQuota) Remaining() int64 { func (b *BackendQuota) Remaining() int64 {
return b.maxBackendBytes - b.be.Size() return b.maxBackendBytes - b.be.Size()
} }
func maxAllowedOverflowBytes(maxBackendBytes int64) int64 {
allow := maxBackendBytes * 10 / 100
if allow > MaxAllowedOverflowQuotaBytes {
allow = MaxAllowedOverflowQuotaBytes
}
return allow
}

View File

@ -44,6 +44,7 @@ var putCmd = &cobra.Command{
var ( var (
keySize int keySize int
valSize int valSize int
deltaValSize int
putTotal int putTotal int
putRate int putRate int
@ -61,6 +62,7 @@ func init() {
RootCmd.AddCommand(putCmd) RootCmd.AddCommand(putCmd)
putCmd.Flags().IntVar(&keySize, "key-size", 8, "Key size of put request") putCmd.Flags().IntVar(&keySize, "key-size", 8, "Key size of put request")
putCmd.Flags().IntVar(&valSize, "val-size", 8, "Value size of put request") putCmd.Flags().IntVar(&valSize, "val-size", 8, "Value size of put request")
putCmd.Flags().IntVar(&deltaValSize, "delta-val-size", 0, "Delta of value size of put request")
putCmd.Flags().IntVar(&putRate, "rate", 0, "Maximum puts per second (0 is no limit)") putCmd.Flags().IntVar(&putRate, "rate", 0, "Maximum puts per second (0 is no limit)")
putCmd.Flags().IntVar(&putTotal, "total", 10000, "Total number of put requests") putCmd.Flags().IntVar(&putTotal, "total", 10000, "Total number of put requests")
@ -83,7 +85,7 @@ func putFunc(cmd *cobra.Command, _ []string) {
} }
limit := rate.NewLimiter(rate.Limit(putRate), 1) limit := rate.NewLimiter(rate.Limit(putRate), 1)
clients := mustCreateClients(totalClients, totalConns) clients := mustCreateClients(totalClients, totalConns)
k, v := make([]byte, keySize), string(mustRandBytes(valSize)) k, v := make([]byte, keySize), string(mustRandBytes(valSize+deltaValSize))
bar = pb.New(putTotal) bar = pb.New(putTotal)
bar.Start() bar.Start()
@ -104,6 +106,7 @@ func putFunc(cmd *cobra.Command, _ []string) {
}(clients[i]) }(clients[i])
} }
baseValSize := valSize - deltaValSize
go func() { go func() {
for i := 0; i < putTotal; i++ { for i := 0; i < putTotal; i++ {
if seqKeys { if seqKeys {
@ -111,7 +114,11 @@ func putFunc(cmd *cobra.Command, _ []string) {
} else { } else {
binary.PutVarint(k, int64(rand.Intn(keySpaceSize))) binary.PutVarint(k, int64(rand.Intn(keySpaceSize)))
} }
requests <- v3.OpPut(string(k), v) deltaV := v
if deltaValSize > 0 {
deltaV = v[:baseValSize+rand.Intn(2*deltaValSize)]
}
requests <- v3.OpPut(string(k), deltaV)
} }
close(requests) close(requests)
}() }()

View File

@ -15,6 +15,7 @@
package cmd package cmd
import ( import (
"math/rand"
"sync" "sync"
"time" "time"
@ -58,6 +59,8 @@ var (
) )
func init() { func init() {
rand.Seed(time.Now().UnixNano())
RootCmd.PersistentFlags().StringSliceVar(&endpoints, "endpoints", []string{"127.0.0.1:2379"}, "gRPC endpoints") RootCmd.PersistentFlags().StringSliceVar(&endpoints, "endpoints", []string{"127.0.0.1:2379"}, "gRPC endpoints")
RootCmd.PersistentFlags().UintVar(&totalConns, "conns", 1, "Total number of gRPC connections") RootCmd.PersistentFlags().UintVar(&totalConns, "conns", 1, "Total number of gRPC connections")
RootCmd.PersistentFlags().UintVar(&totalClients, "clients", 1, "Total number of gRPC clients") RootCmd.PersistentFlags().UintVar(&totalClients, "clients", 1, "Total number of gRPC clients")

View File

@ -19,6 +19,7 @@ import (
"encoding/binary" "encoding/binary"
"fmt" "fmt"
"math" "math"
"math/rand"
"os" "os"
"time" "time"
@ -48,6 +49,7 @@ func init() {
RootCmd.AddCommand(txnPutCmd) RootCmd.AddCommand(txnPutCmd)
txnPutCmd.Flags().IntVar(&keySize, "key-size", 8, "Key size of txn put") txnPutCmd.Flags().IntVar(&keySize, "key-size", 8, "Key size of txn put")
txnPutCmd.Flags().IntVar(&valSize, "val-size", 8, "Value size of txn put") txnPutCmd.Flags().IntVar(&valSize, "val-size", 8, "Value size of txn put")
txnPutCmd.Flags().IntVar(&deltaValSize, "delta-val-size", 0, "Delta of value size of put request")
txnPutCmd.Flags().IntVar(&txnPutOpsPerTxn, "txn-ops", 1, "Number of puts per txn") txnPutCmd.Flags().IntVar(&txnPutOpsPerTxn, "txn-ops", 1, "Number of puts per txn")
txnPutCmd.Flags().IntVar(&txnPutRate, "rate", 0, "Maximum txns per second (0 is no limit)") txnPutCmd.Flags().IntVar(&txnPutRate, "rate", 0, "Maximum txns per second (0 is no limit)")
@ -73,7 +75,7 @@ func txnPutFunc(_ *cobra.Command, _ []string) {
} }
limit := rate.NewLimiter(rate.Limit(txnPutRate), 1) limit := rate.NewLimiter(rate.Limit(txnPutRate), 1)
clients := mustCreateClients(totalClients, totalConns) clients := mustCreateClients(totalClients, totalConns)
k, v := make([]byte, keySize), string(mustRandBytes(valSize)) k, v := make([]byte, keySize), string(mustRandBytes(valSize+deltaValSize))
bar = pb.New(txnPutTotal) bar = pb.New(txnPutTotal)
bar.Start() bar.Start()
@ -93,12 +95,18 @@ func txnPutFunc(_ *cobra.Command, _ []string) {
}(clients[i]) }(clients[i])
} }
baseValSize := valSize - deltaValSize
go func() { go func() {
for i := 0; i < txnPutTotal; i++ { for i := 0; i < txnPutTotal; i++ {
ops := make([]v3.Op, txnPutOpsPerTxn) ops := make([]v3.Op, txnPutOpsPerTxn)
for j := 0; j < txnPutOpsPerTxn; j++ { for j := 0; j < txnPutOpsPerTxn; j++ {
binary.PutVarint(k, int64(((i*txnPutOpsPerTxn)+j)%keySpaceSize)) binary.PutVarint(k, int64(((i*txnPutOpsPerTxn)+j)%keySpaceSize))
ops[j] = v3.OpPut(string(k), v)
deltaV := v
if deltaValSize > 0 {
deltaV = v[:baseValSize+rand.Intn(2*deltaValSize)]
}
ops[j] = v3.OpPut(string(k), deltaV)
} }
requests <- ops requests <- ops
} }