github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner/gc_manager.go (about) 1 // Copyright 2021 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 owner 15 16 import ( 17 "math" 18 "time" 19 20 "github.com/pingcap/errors" 21 "github.com/pingcap/failpoint" 22 "github.com/pingcap/log" 23 "github.com/pingcap/ticdc/cdc/model" 24 "github.com/pingcap/ticdc/pkg/config" 25 cdcContext "github.com/pingcap/ticdc/pkg/context" 26 cerror "github.com/pingcap/ticdc/pkg/errors" 27 "github.com/pingcap/tidb/store/tikv/oracle" 28 "go.uber.org/zap" 29 ) 30 31 const ( 32 // cdcServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. 33 cdcServiceSafePointID = "ticdc" 34 pdTimeUpdateInterval = 10 * time.Minute 35 ) 36 37 // gcSafepointUpdateInterval is the minimum interval that CDC can update gc safepoint 38 var gcSafepointUpdateInterval = 1 * time.Minute 39 40 // GcManager is an interface for gc manager 41 type GcManager interface { 42 updateGCSafePoint(ctx cdcContext.Context, state *model.GlobalReactorState) error 43 currentTimeFromPDCached(ctx cdcContext.Context) (time.Time, error) 44 checkStaleCheckpointTs(ctx cdcContext.Context, checkpointTs model.Ts) error 45 } 46 47 type gcManager struct { 48 gcTTL int64 49 50 lastUpdatedTime time.Time 51 lastSucceededTime time.Time 52 lastSafePointTs uint64 53 isTiCDCBlockGC bool 54 55 pdPhysicalTimeCache time.Time 56 lastUpdatedPdTime time.Time 57 } 58 59 func newGCManager() *gcManager { 60 serverConfig := config.GetGlobalServerConfig() 61 failpoint.Inject("InjectGcSafepointUpdateInterval", func(val failpoint.Value) { 62 gcSafepointUpdateInterval = time.Duration(val.(int) * int(time.Millisecond)) 63 }) 64 return &gcManager{ 65 lastSucceededTime: time.Now(), 66 gcTTL: serverConfig.GcTTL, 67 } 68 } 69 70 func (m *gcManager) updateGCSafePoint(ctx cdcContext.Context, state *model.GlobalReactorState) error { 71 if time.Since(m.lastUpdatedTime) < gcSafepointUpdateInterval { 72 return nil 73 } 74 minCheckpointTs := uint64(math.MaxUint64) 75 for _, cfState := range state.Changefeeds { 76 if cfState.Info == nil { 77 continue 78 } 79 switch cfState.Info.State { 80 case model.StateNormal, model.StateStopped, model.StateError: 81 default: 82 continue 83 } 84 checkpointTs := cfState.Info.GetCheckpointTs(cfState.Status) 85 if minCheckpointTs > checkpointTs { 86 minCheckpointTs = checkpointTs 87 } 88 } 89 m.lastUpdatedTime = time.Now() 90 91 actual, err := ctx.GlobalVars().PDClient.UpdateServiceGCSafePoint(ctx, cdcServiceSafePointID, m.gcTTL, minCheckpointTs) 92 if err != nil { 93 log.Warn("updateGCSafePoint failed", 94 zap.Uint64("safePointTs", minCheckpointTs), 95 zap.Error(err)) 96 if time.Since(m.lastSucceededTime) >= time.Second*time.Duration(m.gcTTL) { 97 return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) 98 } 99 return nil 100 } 101 failpoint.Inject("InjectActualGCSafePoint", func(val failpoint.Value) { 102 actual = uint64(val.(int)) 103 }) 104 if actual == minCheckpointTs { 105 log.Info("update gc safe point success", zap.Uint64("gcSafePointTs", minCheckpointTs)) 106 } 107 if actual > minCheckpointTs { 108 log.Warn("update gc safe point failed, the gc safe point is larger than checkpointTs", zap.Uint64("actual", actual), zap.Uint64("checkpointTs", minCheckpointTs)) 109 } 110 // if the min checkpoint ts is equal to the current gc safe point, 111 // it means that the service gc safe point set by TiCDC is the min service gc safe point 112 m.isTiCDCBlockGC = actual == minCheckpointTs 113 m.lastSafePointTs = actual 114 m.lastSucceededTime = time.Now() 115 return nil 116 } 117 118 func (m *gcManager) currentTimeFromPDCached(ctx cdcContext.Context) (time.Time, error) { 119 if time.Since(m.lastUpdatedPdTime) <= pdTimeUpdateInterval { 120 return m.pdPhysicalTimeCache, nil 121 } 122 physical, logical, err := ctx.GlobalVars().PDClient.GetTS(ctx) 123 if err != nil { 124 return time.Now(), errors.Trace(err) 125 } 126 m.pdPhysicalTimeCache = oracle.GetTimeFromTS(oracle.ComposeTS(physical, logical)) 127 m.lastUpdatedPdTime = time.Now() 128 return m.pdPhysicalTimeCache, nil 129 } 130 131 func (m *gcManager) checkStaleCheckpointTs(ctx cdcContext.Context, checkpointTs model.Ts) error { 132 if m.isTiCDCBlockGC { 133 pdTime, err := m.currentTimeFromPDCached(ctx) 134 if err != nil { 135 return errors.Trace(err) 136 } 137 if pdTime.Sub(oracle.GetTimeFromTS(checkpointTs)) > time.Duration(m.gcTTL)*time.Second { 138 return cerror.ErrGCTTLExceeded.GenWithStackByArgs(checkpointTs, ctx.ChangefeedVars().ID) 139 } 140 } else { 141 // if `isTiCDCBlockGC` is false, it means there is another service gc point less than the min checkpoint ts. 142 if checkpointTs < m.lastSafePointTs { 143 return cerror.ErrSnapshotLostByGC.GenWithStackByArgs(checkpointTs, m.lastSafePointTs) 144 } 145 } 146 return nil 147 }