Files
tidb/pkg/objstore/batch.go

187 lines
5.3 KiB
Go

// Copyright 2026 PingCAP, Inc.
//
// 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 objstore
import (
"context"
"encoding/json"
"fmt"
"io"
"os"
"sync"
"github.com/pingcap/errors"
berrors "github.com/pingcap/tidb/br/pkg/errors"
"github.com/pingcap/tidb/pkg/objstore/objectio"
"github.com/pingcap/tidb/pkg/objstore/storeapi"
"go.uber.org/multierr"
)
// Effect is an side effect that happens in the batch storage.
type Effect any
// EffPut is the side effect of a call to `WriteFile`.
type EffPut struct {
File string `json:"file"`
Content []byte `json:"content"`
}
// EffDeleteFiles is the side effect of a call to `DeleteFiles`.
type EffDeleteFiles struct {
Files []string `json:"files"`
}
// EffDeleteFile is the side effect of a call to `DeleteFile`.
type EffDeleteFile string
// EffRename is the side effect of a call to `Rename`.
type EffRename struct {
From string `json:"from"`
To string `json:"to"`
}
// JSONEffects converts a slices of effects into json.
// The json will be a tagged union: `{"type": $go_type_name, "effect": $effect}`
func JSONEffects(es []Effect, output io.Writer) error {
type Typed struct {
Type string `json:"type"`
Eff Effect `json:"effect"`
}
out := make([]Typed, 0, len(es))
for _, eff := range es {
out = append(out, Typed{
Type: fmt.Sprintf("%T", eff),
Eff: eff,
})
}
return json.NewEncoder(output).Encode(out)
}
// SaveJSONEffectsToTmp save to tmp.
func SaveJSONEffectsToTmp(es []Effect) (string, error) {
// Save the json to a subdir so user can redirect the output path by symlinking...
tmp, err := os.CreateTemp(os.TempDir(), "br-effects-*.json")
if err != nil {
return "", err
}
if err := JSONEffects(es, tmp); err != nil {
return "", err
}
return tmp.Name(), nil
}
// Batched is a wrapper of an external storage that suspends all write operations ("effects").
// If `Close()` without calling `Commit()`, nothing will happen in the underlying external storage.
// In that case, we have done a "dry run".
//
// You may use `ReadOnlyEffects()` to get the history of the effects.
// But don't modify the returned slice!
//
// You may use `Commit()` to execute all suspended effects.
type Batched struct {
storeapi.Storage
effectsMu sync.Mutex
// It will be one of:
// EffPut, EffDeleteFiles, EffDeleteFile, EffRename
effects []Effect
}
// Batch wraps an external storage instance to a batched version.
func Batch(s storeapi.Storage) *Batched {
return &Batched{Storage: s}
}
// ReadOnlyEffects Fetch all effects from the batched storage.
//
// **The returned slice should not be modified.**
func (d *Batched) ReadOnlyEffects() []Effect {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
return d.effects
}
// CleanEffects cleans all suspended effects.
func (d *Batched) CleanEffects() {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
d.effects = nil
}
// DeleteFiles implements the Storage interface.
func (d *Batched) DeleteFiles(ctx context.Context, names []string) error {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
d.effects = append(d.effects, EffDeleteFiles{Files: names})
return nil
}
// DeleteFile implements the Storage interface.
func (d *Batched) DeleteFile(ctx context.Context, name string) error {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
d.effects = append(d.effects, EffDeleteFile(name))
return nil
}
// WriteFile implements the Storage interface.
func (d *Batched) WriteFile(ctx context.Context, name string, data []byte) error {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
d.effects = append(d.effects, EffPut{File: name, Content: data})
return nil
}
// Rename implements the Storage interface.
func (d *Batched) Rename(ctx context.Context, oldName, newName string) error {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
d.effects = append(d.effects, EffRename{From: oldName, To: newName})
return nil
}
// Create implements the Storage interface.
func (d *Batched) Create(ctx context.Context, path string, option *storeapi.WriterOption) (objectio.Writer, error) {
return nil, errors.Annotatef(berrors.ErrStorageUnknown, "ExternalStorage.Create isn't allowed in batch mode for now.")
}
// Commit performs all effects recorded so long in the REAL external storage.
// This will cleanup all of the suspended effects.
func (d *Batched) Commit(ctx context.Context) error {
d.effectsMu.Lock()
defer d.effectsMu.Unlock()
var err error
for _, eff := range d.effects {
switch e := eff.(type) {
case EffPut:
err = multierr.Combine(d.Storage.WriteFile(ctx, e.File, e.Content), err)
case EffDeleteFiles:
err = multierr.Combine(d.Storage.DeleteFiles(ctx, e.Files), err)
case EffDeleteFile:
err = multierr.Combine(d.Storage.DeleteFile(ctx, string(e)), err)
case EffRename:
err = multierr.Combine(d.Storage.Rename(ctx, e.From, e.To), err)
default:
return errors.Annotatef(berrors.ErrStorageUnknown, "Unknown effect type %T", eff)
}
}
d.effects = nil
return nil
}