github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sourcemanager/sorter/factory/pebble.go (about)

     1  // Copyright 2022 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package factory
    15  
    16  import (
    17  	"fmt"
    18  	"strconv"
    19  	"sync/atomic"
    20  	"time"
    21  
    22  	"github.com/cockroachdb/pebble"
    23  	"github.com/pingcap/log"
    24  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter"
    25  	epebble "github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter/pebble"
    26  	"github.com/pingcap/tiflow/pkg/config"
    27  	"go.uber.org/zap"
    28  )
    29  
    30  func createPebbleDBs(
    31  	dir string, cfg *config.DBConfig,
    32  	memQuotaInBytes uint64,
    33  ) (dbs []*pebble.DB, cache *pebble.Cache, writeStalls []writeStall, err error) {
    34  	dbs = make([]*pebble.DB, 0, cfg.Count)
    35  	writeStalls = make([]writeStall, cfg.Count)
    36  	defer func() {
    37  		if err != nil {
    38  			for _, db := range dbs {
    39  				db.Close()
    40  			}
    41  			dbs = nil
    42  			if cache != nil {
    43  				cache.Unref()
    44  				cache = nil
    45  			}
    46  			writeStalls = nil
    47  		}
    48  	}()
    49  
    50  	cache = pebble.NewCache(int64(memQuotaInBytes))
    51  	tableCache := pebble.NewTableCache(cache, 8, int(cache.MaxSize()))
    52  	for id := 0; id < cfg.Count; id++ {
    53  		ws := writeStalls[id]
    54  		adjust := func(opts *pebble.Options) {
    55  			listener := new(pebble.EventListener)
    56  			*listener = pebble.MakeLoggingEventListener(&pebbleLogger{id: id})
    57  			opts.EventListener = listener
    58  
    59  			opts.EventListener.WriteStallBegin = func(_ pebble.WriteStallBeginInfo) {
    60  				atomic.AddUint64(&ws.counter, 1)
    61  				atomic.CompareAndSwapInt64(&ws.startAt, 0, time.Now().UnixNano())
    62  			}
    63  			opts.EventListener.WriteStallEnd = func() {
    64  				startAt := atomic.SwapInt64(&ws.startAt, 0)
    65  				if startAt != 0 {
    66  					elapsed := time.Since(time.Unix(0, startAt)).Milliseconds()
    67  					atomic.AddInt64(&ws.durInMs, elapsed)
    68  				}
    69  			}
    70  			opts.EventListener.CompactionEnd = func(job pebble.CompactionInfo) {
    71  				idstr := strconv.Itoa(id + 1)
    72  				x := sorter.CompactionDuration().WithLabelValues(idstr)
    73  				x.Observe(job.TotalDuration.Seconds())
    74  			}
    75  		}
    76  
    77  		var db *pebble.DB
    78  		if db, err = epebble.OpenPebble(id, dir, cfg, cache, tableCache, adjust); err != nil {
    79  			log.Error("create pebble fails", zap.String("dir", dir), zap.Int("id", id), zap.Error(err))
    80  			return
    81  		}
    82  		log.Info("create pebble instance success",
    83  			zap.Int("id", id+1),
    84  			zap.Uint64("sharedCacheSize", memQuotaInBytes))
    85  		dbs = append(dbs, db)
    86  	}
    87  	err = tableCache.Unref()
    88  	return
    89  }
    90  
    91  type pebbleLogger struct{ id int }
    92  
    93  var _ pebble.Logger = (*pebbleLogger)(nil)
    94  
    95  func (logger *pebbleLogger) Infof(format string, args ...interface{}) {
    96  	// Do not output low-level pebble log to TiCDC log.
    97  	log.Debug(fmt.Sprintf(format, args...), zap.Int("db", logger.id))
    98  }
    99  
   100  func (logger *pebbleLogger) Fatalf(format string, args ...interface{}) {
   101  	log.Panic(fmt.Sprintf(format, args...), zap.Int("db", logger.id))
   102  }
   103  
   104  type writeStall struct {
   105  	counter uint64
   106  	startAt int64
   107  	durInMs int64
   108  }