github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/buffer/nodemgr.go (about) 1 // Copyright 2022 Matrix Origin 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 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package buffer 16 17 import ( 18 "bytes" 19 "context" 20 "fmt" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/matrixorigin/matrixone/pkg/logutil" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/buffer/base" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 28 ) 29 30 type nodeManager struct { 31 sync.RWMutex 32 sizeLimiter 33 nodes map[any]base.INode 34 evicter IEvictHolder 35 unregistertimes atomic.Int64 36 loadtimes atomic.Int64 37 evicttimes atomic.Int64 38 } 39 40 func NewNodeManager(maxsize uint64, evicter IEvictHolder) *nodeManager { 41 if evicter == nil { 42 evicter = NewSimpleEvictHolder() 43 } 44 mgr := &nodeManager{ 45 sizeLimiter: *newSizeLimiter(maxsize), 46 nodes: make(map[any]base.INode), 47 evicter: evicter, 48 } 49 return mgr 50 } 51 52 func (mgr *nodeManager) String() string { 53 var w bytes.Buffer 54 mgr.RLock() 55 defer mgr.RUnlock() 56 loaded := 0 57 _, _ = w.WriteString(fmt.Sprintf("<nodeManager>[%s][Nodes:%d,LoadTimes:%d,EvictTimes:%d,UnregisterTimes:%d]:", 58 mgr.sizeLimiter.String(), 59 len(mgr.nodes), 60 mgr.loadtimes.Load(), 61 mgr.evicttimes.Load(), 62 mgr.unregistertimes.Load())) 63 for _, node := range mgr.nodes { 64 key := node.Key() 65 _ = w.WriteByte('\n') 66 node.RLock() 67 _, _ = w.WriteString(fmt.Sprintf("\t%v | %s | Size: %d ", 68 key, 69 base.NodeStateString(mgr.nodes[key].GetState()), 70 mgr.nodes[key].Size())) 71 if node.GetState() == base.NodeLoaded { 72 loaded++ 73 } 74 node.RUnlock() 75 } 76 _ = w.WriteByte('\n') 77 _, _ = w.WriteString(fmt.Sprintf("[Load Status: (%d/%d)]", loaded, len(mgr.nodes))) 78 79 return w.String() 80 } 81 82 func (mgr *nodeManager) Count() int { 83 mgr.RLock() 84 defer mgr.RUnlock() 85 return len(mgr.nodes) 86 } 87 88 func (mgr *nodeManager) Add(node base.INode) (err error) { 89 if !node.IsLoaded() { 90 return mgr.RegisterNode(node) 91 } 92 ok := mgr.MakeRoom(node.Size()) 93 if !ok { 94 err = base.ErrNoSpace 95 return 96 } 97 return mgr.RegisterNode(node) 98 } 99 100 func (mgr *nodeManager) RegisterNode(node base.INode) error { 101 key := node.Key() 102 mgr.Lock() 103 defer mgr.Unlock() 104 if _, ok := mgr.nodes[key]; ok { 105 return base.ErrDuplicataNode 106 } 107 mgr.nodes[key] = node 108 return nil 109 } 110 111 func (mgr *nodeManager) UnregisterNode(node base.INode) { 112 mgr.Lock() 113 defer mgr.Unlock() 114 mgr.unregistertimes.Add(1) 115 delete(mgr.nodes, node.Key()) 116 node.Destroy() 117 } 118 119 func (mgr *nodeManager) MakeRoom(size uint64) bool { 120 ok := mgr.sizeLimiter.ApplyQuota(size) 121 nodes := make([]base.IEvictHandle, 0) 122 for !ok { 123 evicted := mgr.evicter.Dequeue() 124 if evicted == nil { 125 return false 126 } 127 128 if !evicted.Unloadable(evicted.Handle) { 129 continue 130 } 131 132 { 133 evicted.Handle.Lock() 134 if evicted.Handle.IsClosed() { 135 evicted.Handle.Unlock() 136 continue 137 } 138 if !evicted.Unloadable(evicted.Handle) { 139 evicted.Handle.Unlock() 140 continue 141 } 142 if !evicted.Handle.Unloadable() { 143 evicted.Handle.Unlock() 144 continue 145 } 146 evicted.Handle.Unload() 147 if evicted.Handle.HardEvictable() { 148 nodes = append(nodes, evicted.Handle) 149 } 150 evicted.Handle.Unlock() 151 } 152 ok = mgr.sizeLimiter.ApplyQuota(size) 153 } 154 for _, enode := range nodes { 155 enode.TryClose() 156 } 157 158 return ok 159 } 160 161 func (mgr *nodeManager) GetNodeByKey(key any) (n base.INode, err error) { 162 mgr.RLock() 163 defer mgr.RUnlock() 164 n = mgr.nodes[key] 165 if n == nil { 166 err = base.ErrNotFound 167 } 168 return 169 } 170 171 func (mgr *nodeManager) PinByKey(key any) (h base.INodeHandle, err error) { 172 n, err := mgr.GetNodeByKey(key) 173 if err != nil { 174 return 175 } 176 h = mgr.Pin(n) 177 return 178 } 179 180 func (mgr *nodeManager) TryPinByKey(key any, timeout time.Duration) (h base.INodeHandle, err error) { 181 n, err := mgr.GetNodeByKey(key) 182 if err != nil { 183 return 184 } 185 return mgr.TryPin(n, timeout) 186 } 187 188 func (mgr *nodeManager) TryPin(node base.INode, timeout time.Duration) (h base.INodeHandle, err error) { 189 h = mgr.Pin(node) 190 if h == nil { 191 times := 0 192 var ctx context.Context 193 var cancel context.CancelFunc 194 if timeout > 0 { 195 ctx, cancel = context.WithTimeout(context.Background(), timeout) 196 defer cancel() 197 } 198 err = common.DoRetry(func() (err error) { 199 times++ 200 h = mgr.Pin(node) 201 if h == nil { 202 err = base.ErrNoSpace 203 } 204 return 205 }, ctx) 206 key := node.Key() 207 logutil.Warnf("DoRetry Pin Node %v Times %d: %v", key, times, err) 208 } 209 return 210 } 211 212 func (mgr *nodeManager) Pin(node base.INode) base.INodeHandle { 213 node.RLock() 214 if node.IsLoaded() { 215 node.Ref() 216 node.RUnlock() 217 return node.MakeHandle() 218 } 219 node.RUnlock() 220 221 node.Lock() 222 defer node.Unlock() 223 if node.IsClosed() { 224 return nil 225 } 226 if node.IsLoaded() { 227 node.Ref() 228 return node.MakeHandle() 229 } 230 ok := mgr.MakeRoom(node.Size()) 231 if !ok { 232 return nil 233 } 234 node.Load() 235 mgr.loadtimes.Add(1) 236 node.Ref() 237 return node.MakeHandle() 238 } 239 240 func (mgr *nodeManager) Unpin(node base.INode) { 241 node.Lock() 242 defer node.Unlock() 243 node.Unref() 244 if node.RefCount() == 0 { 245 toevict := &EvictNode{Handle: node, Iter: node.IncIteration()} 246 mgr.evicter.Enqueue(toevict) 247 mgr.evicttimes.Add(1) 248 } 249 }