github.com/dolthub/dolt/go@v0.40.5-0.20240520175717-68db7794bea6/store/nbs/bs_persister.go (about) 1 // Copyright 2019 Dolthub, 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 // 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 nbs 16 17 import ( 18 "bytes" 19 "context" 20 "errors" 21 "fmt" 22 "io" 23 "time" 24 25 "golang.org/x/sync/errgroup" 26 27 "github.com/dolthub/dolt/go/store/blobstore" 28 "github.com/dolthub/dolt/go/store/chunks" 29 "github.com/dolthub/dolt/go/store/hash" 30 ) 31 32 const ( 33 tableRecordsExt = ".records" 34 tableTailExt = ".tail" 35 ) 36 37 type blobstorePersister struct { 38 bs blobstore.Blobstore 39 blockSize uint64 40 q MemoryQuotaProvider 41 } 42 43 var _ tablePersister = &blobstorePersister{} 44 var _ tableFilePersister = &blobstorePersister{} 45 46 // Persist makes the contents of mt durable. Chunks already present in 47 // |haver| may be dropped in the process. 48 func (bsp *blobstorePersister) Persist(ctx context.Context, mt *memTable, haver chunkReader, stats *Stats) (chunkSource, error) { 49 address, data, chunkCount, err := mt.write(haver, stats) 50 if err != nil { 51 return emptyChunkSource{}, err 52 } else if chunkCount == 0 { 53 return emptyChunkSource{}, nil 54 } 55 name := address.String() 56 57 // persist this table in two parts to facilitate later conjoins 58 records, tail := splitTableParts(data, chunkCount) 59 60 // first write table records and tail (index+footer) as separate blobs 61 eg, ectx := errgroup.WithContext(ctx) 62 eg.Go(func() (err error) { 63 _, err = bsp.bs.Put(ectx, name+tableRecordsExt, int64(len(records)), bytes.NewBuffer(records)) 64 return 65 }) 66 eg.Go(func() (err error) { 67 _, err = bsp.bs.Put(ectx, name+tableTailExt, int64(len(tail)), bytes.NewBuffer(tail)) 68 return 69 }) 70 if err = eg.Wait(); err != nil { 71 return nil, err 72 } 73 74 // then concatenate into a final blob 75 if _, err = bsp.bs.Concatenate(ctx, name, []string{name + tableRecordsExt, name + tableTailExt}); err != nil { 76 return emptyChunkSource{}, err 77 } 78 rdr := &bsTableReaderAt{name, bsp.bs} 79 return newReaderFromIndexData(ctx, bsp.q, data, address, rdr, bsp.blockSize) 80 } 81 82 // ConjoinAll implements tablePersister. 83 func (bsp *blobstorePersister) ConjoinAll(ctx context.Context, sources chunkSources, stats *Stats) (chunkSource, cleanupFunc, error) { 84 var sized []sourceWithSize 85 for _, src := range sources { 86 sized = append(sized, sourceWithSize{src, src.currentSize()}) 87 } 88 89 plan, err := planConjoin(sized, stats) 90 if err != nil { 91 return nil, nil, err 92 } 93 address := nameFromSuffixes(plan.suffixes()) 94 name := address.String() 95 96 // conjoin must contiguously append the chunk records of |sources|, but the raw content 97 // of each source contains a chunk index in the tail. Blobstore does not expose a range 98 // copy (GCP Storage limitation), so we must create sub-objects from each source that 99 // contain only chunk records. We make an effort to store these sub-objects on Persist(), 100 // but we will create them in getRecordsSubObjects if necessary. 101 102 conjoinees := make([]string, 0, len(sources)+1) 103 for _, src := range plan.sources.sws { 104 sub, err := bsp.getRecordsSubObject(ctx, src.source) 105 if err != nil { 106 return nil, nil, err 107 } 108 conjoinees = append(conjoinees, sub) 109 } 110 111 // first concatenate all the sub-objects to create a composite sub-object 112 if _, err = bsp.bs.Concatenate(ctx, name+tableRecordsExt, conjoinees); err != nil { 113 return nil, nil, err 114 } 115 if _, err = blobstore.PutBytes(ctx, bsp.bs, name+tableTailExt, plan.mergedIndex); err != nil { 116 return nil, nil, err 117 } 118 // then concatenate into a final blob 119 if _, err = bsp.bs.Concatenate(ctx, name, []string{name + tableRecordsExt, name + tableTailExt}); err != nil { 120 return emptyChunkSource{}, nil, err 121 } 122 123 cs, err := newBSChunkSource(ctx, bsp.bs, address, plan.chunkCount, bsp.q, stats) 124 return cs, func() {}, err 125 } 126 127 func (bsp *blobstorePersister) getRecordsSubObject(ctx context.Context, cs chunkSource) (name string, err error) { 128 name = cs.hash().String() + tableRecordsExt 129 // first check if we created this sub-object on Persist() 130 ok, err := bsp.bs.Exists(ctx, name) 131 if err != nil { 132 return "", err 133 } else if ok { 134 return name, nil 135 } 136 137 // otherwise create the sub-object from |table| 138 // (requires a round-trip for remote blobstores) 139 cnt, err := cs.count() 140 if err != nil { 141 return "", err 142 } 143 off := tableTailOffset(cs.currentSize(), cnt) 144 l := int64(off) 145 rng := blobstore.NewBlobRange(0, l) 146 147 rdr, _, err := bsp.bs.Get(ctx, cs.hash().String(), rng) 148 if err != nil { 149 return "", err 150 } 151 defer func() { 152 if cerr := rdr.Close(); cerr != nil { 153 err = cerr 154 } 155 }() 156 157 if _, err = bsp.bs.Put(ctx, name, l, rdr); err != nil { 158 return "", err 159 } 160 return name, nil 161 } 162 163 // Open a table named |name|, containing |chunkCount| chunks. 164 func (bsp *blobstorePersister) Open(ctx context.Context, name hash.Hash, chunkCount uint32, stats *Stats) (chunkSource, error) { 165 return newBSChunkSource(ctx, bsp.bs, name, chunkCount, bsp.q, stats) 166 } 167 168 func (bsp *blobstorePersister) Exists(ctx context.Context, name hash.Hash, chunkCount uint32, stats *Stats) (bool, error) { 169 return bsp.bs.Exists(ctx, name.String()) 170 } 171 172 func (bsp *blobstorePersister) PruneTableFiles(ctx context.Context, keeper func() []hash.Hash, t time.Time) error { 173 return nil 174 } 175 176 func (bsp *blobstorePersister) Close() error { 177 return nil 178 } 179 180 func (bsp *blobstorePersister) AccessMode() chunks.ExclusiveAccessMode { 181 return chunks.ExclusiveAccessMode_Shared 182 } 183 184 func (bsp *blobstorePersister) Path() string { 185 return "" 186 } 187 188 func (bsp *blobstorePersister) CopyTableFile(ctx context.Context, r io.Reader, name string, fileSz uint64, chunkCount uint32) error { 189 // sanity check file size 190 if fileSz < indexSize(chunkCount)+footerSize { 191 return fmt.Errorf("table file size %d too small for chunk count %d", fileSz, chunkCount) 192 } 193 194 off := int64(tableTailOffset(fileSz, chunkCount)) 195 lr := io.LimitReader(r, off) 196 197 // check if we can Put concurrently 198 rr, ok := r.(io.ReaderAt) 199 if !ok { 200 // sequentially write chunk records then tail 201 if _, err := bsp.bs.Put(ctx, name+tableRecordsExt, off, lr); err != nil { 202 return err 203 } 204 if _, err := bsp.bs.Put(ctx, name+tableTailExt, int64(fileSz), r); err != nil { 205 return err 206 } 207 } else { 208 // on the push path, we expect to Put concurrently 209 // see BufferedFileByteSink in byte_sink.go 210 eg, ectx := errgroup.WithContext(ctx) 211 eg.Go(func() error { 212 buf := make([]byte, indexSize(chunkCount)+footerSize) 213 if _, err := rr.ReadAt(buf, off); err != nil { 214 return err 215 } 216 _, err := bsp.bs.Put(ectx, name+tableTailExt, int64(len(buf)), bytes.NewBuffer(buf)) 217 return err 218 }) 219 eg.Go(func() error { 220 _, err := bsp.bs.Put(ectx, name+tableRecordsExt, off, lr) 221 return err 222 }) 223 if err := eg.Wait(); err != nil { 224 return err 225 } 226 } 227 228 // finally concatenate into the complete table 229 _, err := bsp.bs.Concatenate(ctx, name, []string{name + tableRecordsExt, name + tableTailExt}) 230 return err 231 } 232 233 type bsTableReaderAt struct { 234 key string 235 bs blobstore.Blobstore 236 } 237 238 func (bsTRA *bsTableReaderAt) Close() error { 239 return nil 240 } 241 242 func (bsTRA *bsTableReaderAt) clone() (tableReaderAt, error) { 243 return bsTRA, nil 244 } 245 246 func (bsTRA *bsTableReaderAt) Reader(ctx context.Context) (io.ReadCloser, error) { 247 rc, _, err := bsTRA.bs.Get(ctx, bsTRA.key, blobstore.AllRange) 248 return rc, err 249 } 250 251 // ReadAtWithStats is the bsTableReaderAt implementation of the tableReaderAt interface 252 func (bsTRA *bsTableReaderAt) ReadAtWithStats(ctx context.Context, p []byte, off int64, stats *Stats) (int, error) { 253 br := blobstore.NewBlobRange(off, int64(len(p))) 254 rc, _, err := bsTRA.bs.Get(ctx, bsTRA.key, br) 255 256 if err != nil { 257 return 0, err 258 } 259 defer rc.Close() 260 261 totalRead := 0 262 for totalRead < len(p) { 263 n, err := rc.Read(p[totalRead:]) 264 265 if err != nil && err != io.EOF { 266 return 0, err 267 } 268 269 totalRead += n 270 271 if err == io.EOF { 272 break 273 } 274 } 275 276 return totalRead, nil 277 } 278 279 func newBSChunkSource(ctx context.Context, bs blobstore.Blobstore, name hash.Hash, chunkCount uint32, q MemoryQuotaProvider, stats *Stats) (cs chunkSource, err error) { 280 index, err := loadTableIndex(ctx, stats, chunkCount, q, func(p []byte) error { 281 rc, _, err := bs.Get(ctx, name.String(), blobstore.NewBlobRange(-int64(len(p)), 0)) 282 if err != nil { 283 return err 284 } 285 defer rc.Close() 286 287 _, err = io.ReadFull(rc, p) 288 if err != nil { 289 return err 290 } 291 292 return nil 293 }) 294 if err != nil { 295 return nil, err 296 } 297 298 if chunkCount != index.chunkCount() { 299 return nil, errors.New("unexpected chunk count") 300 } 301 302 tr, err := newTableReader(index, &bsTableReaderAt{name.String(), bs}, s3BlockSize) 303 if err != nil { 304 _ = index.Close() 305 return nil, err 306 } 307 return &chunkSourceAdapter{tr, name}, nil 308 } 309 310 // splitTableParts separates a table into chunk records and meta data. 311 // 312 // +----------------------+-------+--------+ 313 // table format: | Chunk Record 0 ... N | Index | Footer | 314 // +----------------------+-------+--------+ 315 func splitTableParts(data []byte, count uint32) (records, tail []byte) { 316 o := tableTailOffset(uint64(len(data)), count) 317 records, tail = data[:o], data[o:] 318 return 319 } 320 321 func tableTailOffset(size uint64, count uint32) uint64 { 322 return size - (indexSize(count) + footerSize) 323 }