diff --git a/.github/scripts/prepare_db.sh b/.github/scripts/prepare_db.sh index 58c4b57c505d..aabb3fe0f87b 100755 --- a/.github/scripts/prepare_db.sh +++ b/.github/scripts/prepare_db.sh @@ -1,31 +1,32 @@ -#!/bin/bash -e +#!/bin/bash -e source .github/scripts/start_meta_engine.sh [ -z "$TEST" ] && echo "TEST is not set" && exit 1 -# check port is ready until 60s, sleep 1s for each query -check_port(){ +# check port is ready until 60s, sleep 1s for each query +check_port() { port=$1 echo "check for port:" $port for i in {1..30}; do - sudo lsof -i :$port && echo "port is available: $port after $i sec" && return 0 \ - || (echo "port is not available after $i" && sleep 1) + sudo lsof -i :$port && echo "port is available: $port after $i sec" && return 0 || + (echo "port is not available after $i" && sleep 1) done echo "service not ready on: $port" && exit 1 } -install_mysql(){ +install_mysql() { sudo service mysql start sudo mysql -uroot -proot -e "use mysql;alter user 'root'@'localhost' identified with mysql_native_password by '';" sudo mysql -e "create database dev;" + sudo mysql -e "create database dev2;" check_port 3306 } -install_postgres(){ +install_postgres() { sudo service postgresql start sudo chmod 777 /etc/postgresql/*/main/pg_hba.conf - sudo sed -i "s?local.*all.*postgres.*peer?local all postgres trust?" /etc/postgresql/*/main/pg_hba.conf - sudo sed -i "s?host.*all.*all.*32.*scram-sha-256?host all all 127.0.0.1/32 trust?" /etc/postgresql/*/main/pg_hba.conf - sudo sed -i "s?host.*all.*all.*128.*scram-sha-256?host all all ::1/128 trust?" /etc/postgresql/*/main/pg_hba.conf + sudo sed -i "s?local.*all.*postgres.*peer?local all postgres trust?" /etc/postgresql/*/main/pg_hba.conf + sudo sed -i "s?host.*all.*all.*32.*scram-sha-256?host all all 127.0.0.1/32 trust?" /etc/postgresql/*/main/pg_hba.conf + sudo sed -i "s?host.*all.*all.*128.*scram-sha-256?host all all ::1/128 trust?" /etc/postgresql/*/main/pg_hba.conf cat /etc/postgresql/*/main/pg_hba.conf sudo service postgresql restart psql -c "create user runner superuser;" -U postgres @@ -33,7 +34,7 @@ install_postgres(){ psql -c 'create database test;' -U postgres } -install_etcd(){ +install_etcd() { docker run -d \ -p 3379:2379 \ -p 3380:2380 \ @@ -49,29 +50,29 @@ install_etcd(){ check_port 3380 } -install_keydb(){ +install_keydb() { echo "deb https://download.keydb.dev/open-source-dist $(lsb_release -sc) main" | sudo tee /etc/apt/sources.list.d/keydb.list sudo wget -O /etc/apt/trusted.gpg.d/keydb.gpg https://download.keydb.dev/open-source-dist/keyring.gpg sudo .github/scripts/apt_install.sh keydb - keydb-server --storage-provider flash /tmp/ --port 6378 --bind 127.0.0.1 --daemonize yes + keydb-server --storage-provider flash /tmp/ --port 6378 --bind 127.0.0.1 --daemonize yes keydb-server --port 6377 --bind 127.0.0.1 --daemonize yes - check_port 6377 + check_port 6377 check_port 6378 } -install_minio(){ +install_minio() { docker run -d -p 9000:9000 -p 9001:9001 -e "MINIO_ROOT_USER=testUser" -e "MINIO_ROOT_PASSWORD=testUserPassword" quay.io/minio/minio:RELEASE.2022-01-25T19-56-04Z server /data --console-address ":9001" go install github.com/minio/mc@RELEASE.2022-01-07T06-01-38Z && mc config host add local http://127.0.0.1:9000 testUser testUserPassword && mc mb local/testbucket } -install_fdb(){ +install_fdb() { wget -O /home/travis/.m2/foundationdb-clients_6.3.23-1_amd64.deb https://github.com/apple/foundationdb/releases/download/6.3.23/foundationdb-clients_6.3.23-1_amd64.deb wget -O /home/travis/.m2/foundationdb-server_6.3.23-1_amd64.deb https://github.com/apple/foundationdb/releases/download/6.3.23/foundationdb-server_6.3.23-1_amd64.deb sudo dpkg -i /home/travis/.m2/foundationdb-clients_6.3.23-1_amd64.deb /home/travis/.m2/foundationdb-server_6.3.23-1_amd64.deb check_port 4500 } -install_gluster(){ +install_gluster() { sudo systemctl start glusterd.service mkdir -p /tmp/gluster/gv0 sudo hostname jfstest @@ -80,29 +81,29 @@ install_gluster(){ sudo gluster volume info gv0 } -install_litmus(){ +install_litmus() { wget -O /home/travis/.m2/litmus-0.13.tar.gz http://www.webdav.org/neon/litmus/litmus-0.13.tar.gz tar -zxvf /home/travis/.m2/litmus-0.13.tar.gz -C /home/travis/.m2/ - cd /home/travis/.m2/litmus-0.13/ && ./configure && make && cd - + cd /home/travis/.m2/litmus-0.13/ && ./configure && make && cd - } -install_webdav(){ +install_webdav() { wget -O /home/travis/.m2/rclone-v1.57.0-linux-amd64.zip --no-check-certificate https://downloads.rclone.org/v1.57.0/rclone-v1.57.0-linux-amd64.zip unzip /home/travis/.m2/rclone-v1.57.0-linux-amd64.zip -d /home/travis/.m2/ - nohup /home/travis/.m2/rclone-v1.57.0-linux-amd64/rclone serve webdav local --addr 127.0.0.1:9007 >> rclone.log 2>&1 & + nohup /home/travis/.m2/rclone-v1.57.0-linux-amd64/rclone serve webdav local --addr 127.0.0.1:9007 >>rclone.log 2>&1 & } -prepare_db(){ +prepare_db() { case "$TEST" in "test.meta.core") retry install_tikv install_mysql - ;; + ;; "test.meta.non-core") install_postgres install_etcd install_keydb - ;; + ;; "test.cmd") install_minio install_litmus @@ -116,7 +117,7 @@ prepare_db(){ install_minio install_gluster install_webdav - docker run -d --name sftp -p 2222:22 juicedata/ci-sftp + docker run -d --name sftp -p 2222:22 juicedata/ci-sftp install_etcd .github/scripts/setup-hdfs.sh ;; @@ -127,5 +128,3 @@ prepare_db(){ } prepare_db - - diff --git a/go.mod b/go.mod index 63c0d5f4153a..77b51058f648 100644 --- a/go.mod +++ b/go.mod @@ -75,7 +75,7 @@ require ( golang.org/x/crypto v0.21.0 golang.org/x/net v0.23.0 golang.org/x/oauth2 v0.7.0 - golang.org/x/sync v0.2.0 + golang.org/x/sync v0.9.0 golang.org/x/sys v0.18.0 golang.org/x/term v0.18.0 golang.org/x/text v0.14.0 diff --git a/go.sum b/go.sum index bd8a5877f030..27e0f869a37b 100644 --- a/go.sum +++ b/go.sum @@ -1070,8 +1070,8 @@ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.2.0 h1:PUR+T4wwASmuSTYdKjYHI5TD22Wy5ogLU5qZCOLxBrI= -golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.9.0 h1:fEo0HyrW1GIgZdpbhCRO0PkJajUS5H9IFUztCgEo2jQ= +golang.org/x/sync v0.9.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180622082034-63fc586f45fe/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/pkg/meta/backup.go b/pkg/meta/backup.go new file mode 100644 index 000000000000..0893518afb3a --- /dev/null +++ b/pkg/meta/backup.go @@ -0,0 +1,418 @@ +/* + * JuiceFS, Copyright 2024 Juicedata, 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 meta + +import ( + "context" + "encoding/binary" + "encoding/json" + "fmt" + "io" + "sync" + "unsafe" + + "github.com/juicedata/juicefs/pkg/meta/pb" + "google.golang.org/protobuf/proto" + "google.golang.org/protobuf/reflect/protoreflect" + "google.golang.org/protobuf/reflect/protoregistry" +) + +const ( + BakMagic = 0x747083 + BakVersion = 1 + BakEOS = BakMagic // end of segments +) + +const ( + SegTypeUnknown = iota + SegTypeFormat + SegTypeCounter + SegTypeSustained + SegTypeDelFile + SegTypeAcl + SegTypeXattr + SegTypeQuota + SegTypeStat + SegTypeNode + SegTypeChunk + SegTypeSliceRef + SegTypeEdge + SegTypeParent // for redis/tkv only + SegTypeSymlink + SegTypeMix // for redis/tkv only + SegTypeMax +) + +var ( + SegType2Name map[int]protoreflect.FullName + SegName2Type map[protoreflect.FullName]int +) + +func init() { + SegType2Name = map[int]protoreflect.FullName{ + SegTypeFormat: proto.MessageName(&pb.Format{}), + SegTypeCounter: proto.MessageName(&pb.Counters{}), + SegTypeSustained: proto.MessageName(&pb.SustainedList{}), + SegTypeDelFile: proto.MessageName(&pb.DelFileList{}), + SegTypeSliceRef: proto.MessageName(&pb.SliceRefList{}), + SegTypeAcl: proto.MessageName(&pb.AclList{}), + SegTypeXattr: proto.MessageName(&pb.XattrList{}), + SegTypeQuota: proto.MessageName(&pb.QuotaList{}), + SegTypeStat: proto.MessageName(&pb.StatList{}), + SegTypeNode: proto.MessageName(&pb.NodeList{}), + SegTypeChunk: proto.MessageName(&pb.ChunkList{}), + SegTypeEdge: proto.MessageName(&pb.EdgeList{}), + SegTypeParent: proto.MessageName(&pb.ParentList{}), + SegTypeSymlink: proto.MessageName(&pb.SymlinkList{}), + } + + SegName2Type = make(map[protoreflect.FullName]int) + for k, v := range SegType2Name { + SegName2Type[v] = k + } + + SegType2Name[SegTypeMix] = "kv.Mix" +} + +func CreateMessageByName(name protoreflect.FullName) (proto.Message, error) { + typ, err := protoregistry.GlobalTypes.FindMessageByName(name) + if err != nil { + return nil, fmt.Errorf("failed to find message %s's type: %v", name, err) + } + return typ.New().Interface(), nil +} + +var ErrBakEOF = fmt.Errorf("reach backup EOF") + +// BakFormat: BakSegment... + BakEOF + BakFooter +type BakFormat struct { + Offset uint64 + Footer *BakFooter +} + +func NewBakFormat() *BakFormat { + return &BakFormat{ + Footer: &BakFooter{ + Msg: &pb.Footer{ + Magic: BakMagic, + Version: BakVersion, + Infos: make(map[string]*pb.Footer_SegInfo), + }, + }, + } +} + +func (f *BakFormat) WriteSegment(w io.Writer, seg *BakSegment) error { + if seg == nil { + return nil + } + + n, err := seg.Marshal(w) + if err != nil { + return fmt.Errorf("failed to marshal segment %s: %v", seg, err) + } + + name := seg.String() + info, ok := f.Footer.Msg.Infos[name] + if !ok { + info = &pb.Footer_SegInfo{Offset: []uint64{}} + f.Footer.Msg.Infos[name] = info + } + + info.Offset = append(info.Offset, f.Offset) + f.Offset += uint64(n) + return nil +} + +func (f *BakFormat) ReadSegment(r io.Reader) (*BakSegment, error) { + seg := &BakSegment{} + if err := seg.Unmarshal(r); err != nil { + return nil, err + } + return seg, nil +} + +func (f *BakFormat) WriteFooter(w io.Writer) error { + if err := f.writeEOS(w); err != nil { + return err + } + + data, err := f.Footer.Marshal() + if err != nil { + return err + } + n, err := w.Write(data) + if err != nil && n != len(data) { + return fmt.Errorf("failed to write footer: err %v, write len %d, expect len %d", err, n, len(data)) + } + return nil +} + +func (f *BakFormat) writeEOS(w io.Writer) error { + if n, err := w.Write(binary.BigEndian.AppendUint32(nil, BakEOS)); err != nil && n != 4 { + return fmt.Errorf("failed to write EOS: err %w, write len %d, expect len 4", err, n) + } + return nil +} + +func (f *BakFormat) ReadFooter(r io.ReadSeeker) (*BakFooter, error) { + footer := &BakFooter{} + if err := footer.Unmarshal(r); err != nil { + return nil, err + } + if footer.Msg.Magic != BakMagic { + return nil, fmt.Errorf("invalid magic number %d, expect %d", footer.Msg.Magic, BakMagic) + } + return footer, nil +} + +type BakFooter struct { + Msg *pb.Footer + Len uint64 +} + +func (h *BakFooter) Marshal() ([]byte, error) { + data, err := proto.Marshal(h.Msg) + if err != nil { + return nil, fmt.Errorf("failed to marshal footer: %w", err) + } + + h.Len = uint64(len(data)) + data = binary.BigEndian.AppendUint64(data, h.Len) + return data, nil +} + +func (h *BakFooter) Unmarshal(r io.ReadSeeker) error { + lenSize := int64(unsafe.Sizeof(h.Len)) + _, _ = r.Seek(lenSize, io.SeekEnd) + + data := make([]byte, lenSize) + if n, err := r.Read(data); err != nil && n != int(lenSize) { + return fmt.Errorf("failed to read footer length: err %w, read len %d, expect len %d", err, n, lenSize) + } + + h.Len = binary.BigEndian.Uint64(data) + _, _ = r.Seek(int64(h.Len)+lenSize, io.SeekEnd) + data = make([]byte, h.Len) + if n, err := r.Read(data); err != nil && n != int(h.Len) { + return fmt.Errorf("failed to read footer: err %w, read len %d, expect len %d", err, n, h.Len) + } + + if err := proto.Unmarshal(data, h.Msg); err != nil { + return fmt.Errorf("failed to unmarshal footer: %w", err) + } + return nil +} + +type BakSegment struct { + Typ uint32 + Len uint64 + Val proto.Message +} + +func (s *BakSegment) String() string { + return string(proto.MessageName(s.Val).Name()) +} + +func (s *BakSegment) Marshal(w io.Writer) (int, error) { + if s == nil || s.Val == nil { + return 0, fmt.Errorf("segment %s is nil", s) + } + + typ, ok := SegName2Type[proto.MessageName(s.Val)] + if !ok { + return 0, fmt.Errorf("segment type %d is unknown", typ) + } + s.Typ = uint32(typ) + + if err := binary.Write(w, binary.BigEndian, s.Typ); err != nil { + return 0, fmt.Errorf("failed to write segment type %s : %w", s, err) + } + + data, err := proto.Marshal(s.Val) + if err != nil { + return 0, fmt.Errorf("failed to marshal segment message %s : %w", s, err) + } + s.Len = uint64(len(data)) + if err := binary.Write(w, binary.BigEndian, s.Len); err != nil { + return 0, fmt.Errorf("failed to write segment length %s: %w", s, err) + } + + if n, err := w.Write(data); err != nil || n != len(data) { + return 0, fmt.Errorf("failed to write segment data %s: err %w, write len %d, expect len %d", s, err, n, len(data)) + } + + return binary.Size(s.Typ) + binary.Size(s.Len) + len(data), nil +} + +func (s *BakSegment) Unmarshal(r io.Reader) error { + if err := binary.Read(r, binary.BigEndian, &s.Typ); err != nil { + return fmt.Errorf("failed to read segment type: %v", err) + } + + name, ok := SegType2Name[int(s.Typ)] + if !ok { + if s.Typ == BakMagic { + return ErrBakEOF + } + return fmt.Errorf("segment type %d is unknown", s.Typ) + } + + if err := binary.Read(r, binary.BigEndian, &s.Len); err != nil { + return fmt.Errorf("failed to read segment %s length: %v", s, err) + } + + data := make([]byte, s.Len) + n, err := r.Read(data) + if err != nil && n != int(s.Len) { + return fmt.Errorf("failed to read segment value: err %v, read len %d, expect len %d", err, n, s.Len) + } + msg, err := CreateMessageByName(name) + if err != nil { + return fmt.Errorf("failed to create message %s: %v", name, err) + } + if err = proto.Unmarshal(data, msg); err != nil { + return fmt.Errorf("failed to unmarshal segment msg %s: %v", name, err) + } + s.Val = msg + return nil +} + +// Dump Segment + +type DumpOption struct { + KeepSecret bool + CoNum int +} + +func (opt *DumpOption) check() *DumpOption { + if opt == nil { + opt = &DumpOption{} + } + if opt.CoNum < 1 { + opt.CoNum = 1 + } + return opt +} + +type segReleaser interface { + release(msg proto.Message) +} + +type iDumpedSeg interface { + String() string + dump(ctx Context, ch chan *dumpedResult) error + segReleaser +} + +type dumpedSeg struct { + iDumpedSeg + typ int + meta Meta + opt *DumpOption + txn *eTxn +} + +func (s *dumpedSeg) String() string { return string(SegType2Name[s.typ]) } +func (s *dumpedSeg) release(msg proto.Message) {} + +type formatDS struct { + dumpedSeg +} + +func (s *formatDS) dump(ctx Context, ch chan *dumpedResult) error { + f := s.meta.GetFormat() + return dumpResult(ctx, ch, &dumpedResult{s, ConvertFormatToPB(&f, s.opt.KeepSecret)}) +} + +type dumpedBatchSeg struct { + dumpedSeg + pools []*sync.Pool +} + +type dumpedResult struct { + seg segReleaser + msg proto.Message +} + +func dumpResult(ctx context.Context, ch chan *dumpedResult, res *dumpedResult) error { + select { + case <-ctx.Done(): + return ctx.Err() + case ch <- res: + } + return nil +} + +// Load Segment... + +type LoadOption struct { + CoNum int +} + +func (opt *LoadOption) check() { + if opt.CoNum < 1 { + opt.CoNum = 1 + } +} + +type iLoadedSeg interface { + String() string + load(ctx Context, msg proto.Message) error +} + +type loadedSeg struct { + iLoadedSeg + typ int + meta Meta +} + +func (s *loadedSeg) String() string { return string(SegType2Name[s.typ]) } + +// Message Marshal/Unmarshal + +func ConvertFormatToPB(f *Format, keepSecret bool) *pb.Format { + if !keepSecret { + f.RemoveSecret() + } + data, err := json.MarshalIndent(f, "", "") + if err != nil { + logger.Errorf("failed to marshal format %s: %v", f.Name, err) + return nil + } + return &pb.Format{ + Data: data, + } +} + +// transaction + +type txMaxRetryKey struct{} + +type bTxnOption struct { + coNum int + notUsed bool + readOnly bool + maxRetry int + maxStmtRetry int +} + +type eTxn struct { + en engine + opt *bTxnOption + obj interface{} // real transaction object for different engine +} diff --git a/pkg/meta/base.go b/pkg/meta/base.go index 8538cca7a15f..a1ea1aedddf6 100644 --- a/pkg/meta/base.go +++ b/pkg/meta/base.go @@ -21,6 +21,7 @@ import ( "encoding/binary" "encoding/json" "fmt" + "io" "os" "path" "reflect" @@ -38,6 +39,7 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" ) const ( @@ -133,6 +135,11 @@ type engine interface { cacheACLs(ctx Context) error newDirHandler(inode Ino, plus bool, entries []*Entry) DirHandler + + execETxn(ctx Context, txn *eTxn, fn func(ctx Context, txn *eTxn) error) error + buildDumpedSeg(typ int, opt *DumpOption, txn *eTxn) iDumpedSeg + buildLoadedSeg(typ int, opt *LoadOption) iLoadedSeg + prepareLoad(ctx Context, opt *LoadOption) error } type trashSliceScan func(ss []Slice, ts int64) (clean bool, err error) @@ -3071,3 +3078,130 @@ func (h *dirHandler) Close() { h.readOff = 0 h.Unlock() } + +func (m *baseMeta) DumpMetaV2(ctx Context, w io.Writer, opt *DumpOption) error { + opt = opt.check() + + bak := NewBakFormat() + ch := make(chan *dumpedResult, 100) + wg := &sync.WaitGroup{} + wg.Add(1) + go func() { + defer wg.Done() + + txn := &eTxn{ + en: m.en, + opt: &bTxnOption{ + coNum: opt.CoNum, + readOnly: true, + maxRetry: 1, + maxStmtRetry: 3, + }, + } + err := m.en.execETxn(ctx, txn, func(ctx Context, txn *eTxn) error { + for typ := SegTypeFormat; typ < SegTypeMax; typ++ { + seg := m.en.buildDumpedSeg(typ, opt, txn) + if seg != nil { + if err := seg.dump(ctx, ch); err != nil { + return fmt.Errorf("dump %s err: %w", seg, err) + } + } + } + return nil + }) + if err != nil { + ctx.Cancel() + } else { + close(ch) + } + }() + + var res *dumpedResult + for { + select { + case <-ctx.Done(): + wg.Wait() + return ctx.Err() + case res = <-ch: + } + if res == nil { + break + } + if err := bak.WriteSegment(w, &BakSegment{Val: res.msg}); err != nil { + logger.Errorf("write %s err: %v", res.seg, err) + ctx.Cancel() + wg.Wait() + return err + } + res.seg.release(res.msg) + } + + wg.Wait() + return bak.WriteFooter(w) +} + +func (m *baseMeta) LoadMetaV2(ctx Context, r io.Reader, opt *LoadOption) error { + if opt == nil { + opt = &LoadOption{} + } + if err := m.en.prepareLoad(ctx, opt); err != nil { + return err + } + + type task struct { + msg proto.Message + seg iLoadedSeg + } + + var wg sync.WaitGroup + taskCh := make(chan *task, 100) + + workerFunc := func(ctx Context, taskCh <-chan *task) { + defer wg.Done() + var task *task + for { + select { + case <-ctx.Done(): + return + case task = <-taskCh: + } + if task == nil { + break + } + if err := task.seg.load(ctx, task.msg); err != nil { + logger.Errorf("failed to insert %s: %s", task.seg, err) + ctx.Cancel() + return + } + } + } + + for i := 0; i < opt.CoNum; i++ { + wg.Add(1) + go workerFunc(ctx, taskCh) + } + + bak := NewBakFormat() + for { + seg, err := bak.ReadSegment(r) + if err != nil { + if errors.Is(err, ErrBakEOF) { + close(taskCh) + break + } + ctx.Cancel() + wg.Wait() + return err + } + + ls := m.en.buildLoadedSeg(int(seg.Typ), opt) + select { + case <-ctx.Done(): + wg.Wait() + return ctx.Err() + case taskCh <- &task{seg.Val, ls}: + } + } + wg.Wait() + return nil +} diff --git a/pkg/meta/interface.go b/pkg/meta/interface.go index 65b1d1345225..14907236ba60 100644 --- a/pkg/meta/interface.go +++ b/pkg/meta/interface.go @@ -455,6 +455,9 @@ type Meta interface { DumpMeta(w io.Writer, root Ino, threads int, keepSecret, fast, skipTrash bool) error LoadMeta(r io.Reader) error + DumpMetaV2(ctx Context, w io.Writer, opt *DumpOption) (err error) + LoadMetaV2(ctx Context, r io.Reader, opt *LoadOption) error + // getBase return the base engine. getBase() *baseMeta InitMetrics(registerer prometheus.Registerer) diff --git a/pkg/meta/load_dump_test.go b/pkg/meta/load_dump_test.go index 2c3f41b961b4..fed5c18d1c71 100644 --- a/pkg/meta/load_dump_test.go +++ b/pkg/meta/load_dump_test.go @@ -25,7 +25,10 @@ import ( "path" "strings" "testing" + "time" + aclAPI "github.com/juicedata/juicefs/pkg/acl" + "github.com/sirupsen/logrus" "golang.org/x/text/encoding/simplifiedchinese" "golang.org/x/text/transform" ) @@ -64,7 +67,7 @@ func TestEscape(t *testing.T) { s := escape(string(v)) t.Log("escape value: ", s) r := unescape(s) - if bytes.Compare(r, v) != 0 { + if !bytes.Equal(r, v) { t.Fatalf("expected %v, but got %v", v, r) } } @@ -88,20 +91,7 @@ func GbkToUtf8(s []byte) ([]byte, error) { return d, nil } -func testLoad(t *testing.T, uri, fname string) Meta { - m := NewClient(uri, nil) - if err := m.Reset(); err != nil { - t.Fatalf("reset meta: %s", err) - } - fp, err := os.Open(fname) - if err != nil { - t.Fatalf("open file: %s", fname) - } - defer fp.Close() - if err = m.LoadMeta(fp); err != nil { - t.Fatalf("load meta: %s", err) - } - +func checkMeta(t *testing.T, m Meta) { ctx := Background var entries []*Entry if st := m.Readdir(ctx, 1, 1, &entries); st != 0 { @@ -177,6 +167,48 @@ func testLoad(t *testing.T, uri, fname string) Meta { t.Fatalf("expect the flags euqal 128, but actual is: %d", attr.Flags) } + if attr.AccessACL == 0 || attr.DefaultACL == 0 { + t.Fatalf("expect ACL not 0, but actual is: %d, %d", attr.AccessACL, attr.DefaultACL) + } + + ar := &aclAPI.Rule{} + if st := m.GetFacl(ctx, 2, aclAPI.TypeAccess, ar); st != 0 { + t.Fatalf("get access acl: %s", st) + } + ar2 := &aclAPI.Rule{ + Owner: 6, + Group: 4, + Mask: 4, + Other: 4, + NamedUsers: []aclAPI.Entry{ + {Id: 1, Perm: 6}, + {Id: 2, Perm: 7}, + }, + NamedGroups: nil, + } + if !bytes.Equal(ar.Encode(), ar2.Encode()) { + t.Fatalf("access acl: %v != %v", ar, ar2) + } + + dr := &aclAPI.Rule{} + if st := m.GetFacl(ctx, 2, aclAPI.TypeDefault, dr); st != 0 { + t.Fatalf("get default acl: %s", st) + } + dr2 := &aclAPI.Rule{ + Owner: 7, + Group: 5, + Mask: 5, + Other: 5, + NamedUsers: nil, + NamedGroups: []aclAPI.Entry{ + {Id: 3, Perm: 6}, + {Id: 4, Perm: 7}, + }, + } + if !bytes.Equal(dr.Encode(), dr2.Encode()) { + t.Fatalf("default acl: %v != %v", dr, dr2) + } + var slices []Slice if st := m.Read(ctx, 2, 0, &slices); st != 0 { t.Fatalf("read chunk: %s", st) @@ -207,7 +239,22 @@ func testLoad(t *testing.T, uri, fname string) Meta { if st := m.GetXattr(ctx, 3, "dk", &value); st != 0 || string(value) != "果汁%25" { t.Fatalf("getxattr: %s %v", st, value) } +} +func testLoad(t *testing.T, uri, fname string) Meta { + m := NewClient(uri, nil) + if err := m.Reset(); err != nil { + t.Fatalf("reset meta: %s", err) + } + fp, err := os.Open(fname) + if err != nil { + t.Fatalf("open file: %s", fname) + } + defer fp.Close() + if err = m.LoadMeta(fp); err != nil { + t.Fatalf("load meta: %s", err) + } + checkMeta(t, m) return m } @@ -288,6 +335,91 @@ func TestLoadDump(t *testing.T) { //skip mutate testLoadDump(t, "tikv", "tikv://127.0.0.1:2379/jfs-load-dump") } +func testDumpV2(t *testing.T, m Meta, result string, opt *DumpOption) { + if opt == nil { + opt = &DumpOption{CoNum: 10, KeepSecret: true} + } + fp, err := os.OpenFile(result, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) + if err != nil { + t.Fatalf("open file %s: %s", result, err) + } + defer fp.Close() + if _, err = m.Load(true); err != nil { + t.Fatalf("load setting: %s", err) + } + if err = m.DumpMetaV2(Background, fp, opt); err != nil { + t.Fatalf("dump meta: %s", err) + } + fp.Sync() +} + +func testLoadV2(t *testing.T, uri, fname string) Meta { + m := NewClient(uri, nil) + if err := m.Reset(); err != nil { + t.Fatalf("reset meta: %s", err) + } + fp, err := os.Open(fname) + if err != nil { + t.Fatalf("open file: %s", fname) + } + defer fp.Close() + if err = m.LoadMetaV2(Background, fp, &LoadOption{CoNum: 10}); err != nil { + t.Fatalf("load meta: %s", err) + } + if _, err := m.Load(true); err != nil { + t.Fatalf("load setting: %s", err) + } + checkMeta(t, m) + return m +} + +func testLoadDumpV2(t *testing.T, name, addr1, addr2 string) { + t.Run("Metadata Engine: "+name, func(t *testing.T) { + start := time.Now() + m := testLoad(t, addr1, sampleFile) + t.Logf("load meta: %v", time.Since(start)) + start = time.Now() + testDumpV2(t, m, fmt.Sprintf("%s.dump", name), nil) + m.Shutdown() + t.Logf("dump meta v2: %v", time.Since(start)) + start = time.Now() + m = testLoadV2(t, addr2, fmt.Sprintf("%s.dump", name)) + m.Shutdown() + t.Logf("load meta v2: %v", time.Since(start)) + }) +} + +func testLoadOtherEngine(t *testing.T, src, dst, dstAddr string) { + t.Run(fmt.Sprintf("Load %s to %s", src, dst), func(t *testing.T) { + m := testLoadV2(t, dstAddr, fmt.Sprintf("%s.dump", src)) + m.Shutdown() + }) +} + +func TestLoadDumpV2(t *testing.T) { + logger.SetLevel(logrus.DebugLevel) + + engines := map[string][]string{ + "mysql": {"mysql://root:@/dev", "mysql://root:@/dev2"}, + // "redis": {"redis://127.0.0.1:6379/2", "redis://127.0.0.1:6379/3"}, + // "tikv": {"tikv://127.0.0.1:2379/jfs-load-dump-1", "tikv://127.0.0.1:2379/jfs-load-dump-2"}, + } + + for name, addrs := range engines { + testLoadDumpV2(t, name, addrs[0], addrs[1]) + testSecretAndTrash(t, addrs[0], addrs[1]) + } + + for src := range engines { + for dst, dstAddr := range engines { + if src == dst { + continue + } + testLoadOtherEngine(t, src, dst, dstAddr[1]) + } + } +} + func TestLoadDumpSlow(t *testing.T) { //skip mutate if os.Getenv("SKIP_NON_CORE") == "true" { t.Skipf("skip non-core test") @@ -319,3 +451,122 @@ func TestLoadDump_MemKV(t *testing.T) { testLoadSub(t, "memkv://user:pass@test/jfs", subSampleFile) }) } + +func testSecretAndTrash(t *testing.T, addr, addr2 string) { + m := testLoad(t, addr, sampleFile) + testDumpV2(t, m, "sqlite-secret.dump", &DumpOption{CoNum: 10, KeepSecret: true}) + m2 := testLoadV2(t, addr2, "sqlite-secret.dump") + if m2.GetFormat().EncryptKey != m.GetFormat().EncryptKey { + t.Fatalf("encrypt key not valid: %s", m2.GetFormat().EncryptKey) + } + + testDumpV2(t, m, "sqlite-non-secret.dump", &DumpOption{CoNum: 10, KeepSecret: false}) + m2.Reset() + m2 = testLoadV2(t, addr2, "sqlite-non-secret.dump") + if m2.GetFormat().EncryptKey != "removed" { + t.Fatalf("encrypt key not valid: %s", m2.GetFormat().EncryptKey) + } + + // trash + trashs := map[Ino]uint64{ + 27: 11, + 29: 10485760, + } + cnt := 0 + m2.getBase().scanTrashFiles(Background, func(inode Ino, size uint64, ts time.Time) (clean bool, err error) { + cnt++ + if tSize, ok := trashs[inode]; !ok || size != tSize { + t.Fatalf("trash file: %d %d", inode, size) + } + return false, nil + }) + if cnt != len(trashs) { + t.Fatalf("trash count: %d", cnt) + } +} + +/* +func BenchmarkLoadDumpV2(b *testing.B) { + logrus.SetLevel(logrus.DebugLevel) + b.ReportAllocs() + engines := map[string]string{ + "mysql": "mysql://root:@/dev", + "redis": "redis://127.0.0.1:6379/2", + "tikv": "tikv://127.0.0.1:2379/jfs-load-dump-1", + } + + sample := "../../1M_files_in_one_dir.dump" + for name, addr := range engines { + m := NewClient(addr, nil) + defer func() { + m.Reset() + m.Shutdown() + }() + b.Run("Load "+name, func(b *testing.B) { + if err := m.Reset(); err != nil { + b.Fatalf("reset meta: %s", err) + } + fp, err := os.Open(sample) + if err != nil { + b.Fatalf("open file: %s", sample) + } + defer fp.Close() + + b.ResetTimer() + if err = m.LoadMeta(fp); err != nil { + b.Fatalf("load meta: %s", err) + } + }) + + b.Run("Dump "+name, func(b *testing.B) { + path := fmt.Sprintf("%s.v1.dump", name) + fp, err := os.OpenFile(path, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) + if err != nil { + b.Fatalf("open file %s: %s", path, err) + } + defer fp.Close() + if _, err = m.Load(true); err != nil { + b.Fatalf("load setting: %s", err) + } + + b.ResetTimer() + if err = m.DumpMeta(fp, RootInode, 10, true, true, false); err != nil { + b.Fatalf("dump meta: %s", err) + } + fp.Sync() + }) + + b.Run("DumpV2 "+name, func(b *testing.B) { + path := fmt.Sprintf("%s.v2.dump", name) + fp, err := os.OpenFile(path, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) + if err != nil { + b.Fatalf("open file %s: %s", path, err) + } + defer fp.Close() + + b.ResetTimer() + if err = m.DumpMetaV2(Background, fp, &DumpOption{CoNum: 10}); err != nil { + b.Fatalf("dump meta: %s", err) + } + fp.Sync() + }) + + b.Run("LoadV2 "+name, func(b *testing.B) { + path := fmt.Sprintf("%s.v2.dump", name) + if err := m.Reset(); err != nil { + b.Fatalf("reset meta: %s", err) + } + fp, err := os.Open(path) + if err != nil { + b.Fatalf("open file: %s", path) + } + defer fp.Close() + + b.ResetTimer() + if err = m.LoadMetaV2(Background, fp, &LoadOption{CoNum: 10}); err != nil { + b.Fatalf("load meta: %s", err) + } + }) + } +} +*/ diff --git a/pkg/meta/metadata.sample b/pkg/meta/metadata.sample index a8f5fcd13d98..bf6766103e29 100644 --- a/pkg/meta/metadata.sample +++ b/pkg/meta/metadata.sample @@ -79,6 +79,8 @@ "f1": { "attr": {"inode":2,"flags":128,"type":"regular","mode":420,"uid":501,"gid":20,"atime":1623746580,"mtime":1623746661,"ctime":1623746661,"atimensec":219686444,"mtimensec":219686444,"ctimensec":219686444,"nlink":1,"length":24}, "xattrs": [{"name":"k","value":"v"}], + "posix_acl_access": {"owner":6,"group":4,"other":4,"mask":4,"users":[{"id":1,"perm":6},{"id":2,"perm":7}],"groups":null}, + "posix_acl_default": {"owner":7,"group":5,"other":5,"mask":5,"users":null,"groups":[{"id":3,"perm":6},{"id":4,"perm":7}]}, "chunks": [{"index":0,"slices":[{"id":1,"size":6,"len":6},{"id":2,"size":12,"len":12},{"id":4,"size":24,"len":24}]}] }, "l1": { diff --git a/pkg/meta/pb/backup.pb.go b/pkg/meta/pb/backup.pb.go new file mode 100644 index 000000000000..be94006a80ef --- /dev/null +++ b/pkg/meta/pb/backup.pb.go @@ -0,0 +1,1750 @@ +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.35.1 +// protoc v3.12.4 +// source: pkg/meta/pb/backup.proto + +package pb + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +type Format struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"` // meta.Format's json format +} + +func (x *Format) Reset() { + *x = Format{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Format) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Format) ProtoMessage() {} + +func (x *Format) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[0] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Format.ProtoReflect.Descriptor instead. +func (*Format) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{0} +} + +func (x *Format) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +type Counters struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + UsedSpace int64 `protobuf:"varint,1,opt,name=usedSpace,proto3" json:"usedSpace,omitempty"` + UsedInodes int64 `protobuf:"varint,2,opt,name=usedInodes,proto3" json:"usedInodes,omitempty"` + NextInode int64 `protobuf:"varint,3,opt,name=nextInode,proto3" json:"nextInode,omitempty"` + NextChunk int64 `protobuf:"varint,4,opt,name=nextChunk,proto3" json:"nextChunk,omitempty"` + NextSession int64 `protobuf:"varint,5,opt,name=nextSession,proto3" json:"nextSession,omitempty"` + NextTrash int64 `protobuf:"varint,6,opt,name=nextTrash,proto3" json:"nextTrash,omitempty"` +} + +func (x *Counters) Reset() { + *x = Counters{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Counters) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Counters) ProtoMessage() {} + +func (x *Counters) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[1] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Counters.ProtoReflect.Descriptor instead. +func (*Counters) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{1} +} + +func (x *Counters) GetUsedSpace() int64 { + if x != nil { + return x.UsedSpace + } + return 0 +} + +func (x *Counters) GetUsedInodes() int64 { + if x != nil { + return x.UsedInodes + } + return 0 +} + +func (x *Counters) GetNextInode() int64 { + if x != nil { + return x.NextInode + } + return 0 +} + +func (x *Counters) GetNextChunk() int64 { + if x != nil { + return x.NextChunk + } + return 0 +} + +func (x *Counters) GetNextSession() int64 { + if x != nil { + return x.NextSession + } + return 0 +} + +func (x *Counters) GetNextTrash() int64 { + if x != nil { + return x.NextTrash + } + return 0 +} + +type Sustained struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Sid uint64 `protobuf:"varint,1,opt,name=sid,proto3" json:"sid,omitempty"` + Inodes []uint64 `protobuf:"varint,2,rep,packed,name=inodes,proto3" json:"inodes,omitempty"` +} + +func (x *Sustained) Reset() { + *x = Sustained{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[2] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Sustained) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Sustained) ProtoMessage() {} + +func (x *Sustained) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[2] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Sustained.ProtoReflect.Descriptor instead. +func (*Sustained) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{2} +} + +func (x *Sustained) GetSid() uint64 { + if x != nil { + return x.Sid + } + return 0 +} + +func (x *Sustained) GetInodes() []uint64 { + if x != nil { + return x.Inodes + } + return nil +} + +type SustainedList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Sustained `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *SustainedList) Reset() { + *x = SustainedList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[3] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SustainedList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SustainedList) ProtoMessage() {} + +func (x *SustainedList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[3] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SustainedList.ProtoReflect.Descriptor instead. +func (*SustainedList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{3} +} + +func (x *SustainedList) GetList() []*Sustained { + if x != nil { + return x.List + } + return nil +} + +type DelFile struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Length uint64 `protobuf:"varint,2,opt,name=length,proto3" json:"length,omitempty"` + Expire int64 `protobuf:"varint,3,opt,name=expire,proto3" json:"expire,omitempty"` +} + +func (x *DelFile) Reset() { + *x = DelFile{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[4] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DelFile) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DelFile) ProtoMessage() {} + +func (x *DelFile) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[4] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DelFile.ProtoReflect.Descriptor instead. +func (*DelFile) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{4} +} + +func (x *DelFile) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *DelFile) GetLength() uint64 { + if x != nil { + return x.Length + } + return 0 +} + +func (x *DelFile) GetExpire() int64 { + if x != nil { + return x.Expire + } + return 0 +} + +type DelFileList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*DelFile `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *DelFileList) Reset() { + *x = DelFileList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[5] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *DelFileList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*DelFileList) ProtoMessage() {} + +func (x *DelFileList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[5] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use DelFileList.ProtoReflect.Descriptor instead. +func (*DelFileList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{5} +} + +func (x *DelFileList) GetList() []*DelFile { + if x != nil { + return x.List + } + return nil +} + +type SliceRef struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id uint64 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Size uint32 `protobuf:"varint,2,opt,name=size,proto3" json:"size,omitempty"` + Refs int64 `protobuf:"varint,3,opt,name=refs,proto3" json:"refs,omitempty"` +} + +func (x *SliceRef) Reset() { + *x = SliceRef{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[6] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SliceRef) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SliceRef) ProtoMessage() {} + +func (x *SliceRef) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[6] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SliceRef.ProtoReflect.Descriptor instead. +func (*SliceRef) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{6} +} + +func (x *SliceRef) GetId() uint64 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *SliceRef) GetSize() uint32 { + if x != nil { + return x.Size + } + return 0 +} + +func (x *SliceRef) GetRefs() int64 { + if x != nil { + return x.Refs + } + return 0 +} + +type SliceRefList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*SliceRef `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *SliceRefList) Reset() { + *x = SliceRefList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[7] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SliceRefList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SliceRefList) ProtoMessage() {} + +func (x *SliceRefList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[7] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SliceRefList.ProtoReflect.Descriptor instead. +func (*SliceRefList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{7} +} + +func (x *SliceRefList) GetList() []*SliceRef { + if x != nil { + return x.List + } + return nil +} + +type Acl struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Id uint32 `protobuf:"varint,1,opt,name=id,proto3" json:"id,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` // acl.Rule's binary format +} + +func (x *Acl) Reset() { + *x = Acl{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[8] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Acl) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Acl) ProtoMessage() {} + +func (x *Acl) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[8] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Acl.ProtoReflect.Descriptor instead. +func (*Acl) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{8} +} + +func (x *Acl) GetId() uint32 { + if x != nil { + return x.Id + } + return 0 +} + +func (x *Acl) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +type AclList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Acl `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *AclList) Reset() { + *x = AclList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[9] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *AclList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*AclList) ProtoMessage() {} + +func (x *AclList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[9] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use AclList.ProtoReflect.Descriptor instead. +func (*AclList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{9} +} + +func (x *AclList) GetList() []*Acl { + if x != nil { + return x.List + } + return nil +} + +type Xattr struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Name string `protobuf:"bytes,2,opt,name=name,proto3" json:"name,omitempty"` + Value []byte `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"` +} + +func (x *Xattr) Reset() { + *x = Xattr{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[10] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Xattr) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Xattr) ProtoMessage() {} + +func (x *Xattr) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[10] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Xattr.ProtoReflect.Descriptor instead. +func (*Xattr) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{10} +} + +func (x *Xattr) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Xattr) GetName() string { + if x != nil { + return x.Name + } + return "" +} + +func (x *Xattr) GetValue() []byte { + if x != nil { + return x.Value + } + return nil +} + +type XattrList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Xattr `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *XattrList) Reset() { + *x = XattrList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[11] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *XattrList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*XattrList) ProtoMessage() {} + +func (x *XattrList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[11] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use XattrList.ProtoReflect.Descriptor instead. +func (*XattrList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{11} +} + +func (x *XattrList) GetList() []*Xattr { + if x != nil { + return x.List + } + return nil +} + +type Quota struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + MaxSpace int64 `protobuf:"varint,2,opt,name=maxSpace,proto3" json:"maxSpace,omitempty"` + MaxInodes int64 `protobuf:"varint,3,opt,name=maxInodes,proto3" json:"maxInodes,omitempty"` + UsedSpace int64 `protobuf:"varint,4,opt,name=usedSpace,proto3" json:"usedSpace,omitempty"` + UsedInodes int64 `protobuf:"varint,5,opt,name=usedInodes,proto3" json:"usedInodes,omitempty"` +} + +func (x *Quota) Reset() { + *x = Quota{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[12] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Quota) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Quota) ProtoMessage() {} + +func (x *Quota) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[12] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Quota.ProtoReflect.Descriptor instead. +func (*Quota) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{12} +} + +func (x *Quota) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Quota) GetMaxSpace() int64 { + if x != nil { + return x.MaxSpace + } + return 0 +} + +func (x *Quota) GetMaxInodes() int64 { + if x != nil { + return x.MaxInodes + } + return 0 +} + +func (x *Quota) GetUsedSpace() int64 { + if x != nil { + return x.UsedSpace + } + return 0 +} + +func (x *Quota) GetUsedInodes() int64 { + if x != nil { + return x.UsedInodes + } + return 0 +} + +type QuotaList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Quota `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *QuotaList) Reset() { + *x = QuotaList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *QuotaList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*QuotaList) ProtoMessage() {} + +func (x *QuotaList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[13] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use QuotaList.ProtoReflect.Descriptor instead. +func (*QuotaList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{13} +} + +func (x *QuotaList) GetList() []*Quota { + if x != nil { + return x.List + } + return nil +} + +type Stat struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + DataLength int64 `protobuf:"varint,2,opt,name=dataLength,proto3" json:"dataLength,omitempty"` + UsedSpace int64 `protobuf:"varint,3,opt,name=usedSpace,proto3" json:"usedSpace,omitempty"` + UsedInodes int64 `protobuf:"varint,4,opt,name=usedInodes,proto3" json:"usedInodes,omitempty"` +} + +func (x *Stat) Reset() { + *x = Stat{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Stat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Stat) ProtoMessage() {} + +func (x *Stat) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[14] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Stat.ProtoReflect.Descriptor instead. +func (*Stat) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{14} +} + +func (x *Stat) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Stat) GetDataLength() int64 { + if x != nil { + return x.DataLength + } + return 0 +} + +func (x *Stat) GetUsedSpace() int64 { + if x != nil { + return x.UsedSpace + } + return 0 +} + +func (x *Stat) GetUsedInodes() int64 { + if x != nil { + return x.UsedInodes + } + return 0 +} + +type StatList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Stat `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *StatList) Reset() { + *x = StatList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[15] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *StatList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*StatList) ProtoMessage() {} + +func (x *StatList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[15] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use StatList.ProtoReflect.Descriptor instead. +func (*StatList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{15} +} + +func (x *StatList) GetList() []*Stat { + if x != nil { + return x.List + } + return nil +} + +type Node struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"` // meta.Attr's binary format +} + +func (x *Node) Reset() { + *x = Node{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[16] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Node) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Node) ProtoMessage() {} + +func (x *Node) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[16] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Node.ProtoReflect.Descriptor instead. +func (*Node) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{16} +} + +func (x *Node) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Node) GetData() []byte { + if x != nil { + return x.Data + } + return nil +} + +type NodeList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Node `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *NodeList) Reset() { + *x = NodeList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[17] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *NodeList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*NodeList) ProtoMessage() {} + +func (x *NodeList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[17] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use NodeList.ProtoReflect.Descriptor instead. +func (*NodeList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{17} +} + +func (x *NodeList) GetList() []*Node { + if x != nil { + return x.List + } + return nil +} + +type Edge struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Parent uint64 `protobuf:"varint,1,opt,name=parent,proto3" json:"parent,omitempty"` + Inode uint64 `protobuf:"varint,2,opt,name=inode,proto3" json:"inode,omitempty"` + Name []byte `protobuf:"bytes,3,opt,name=name,proto3" json:"name,omitempty"` + Type uint32 `protobuf:"varint,4,opt,name=type,proto3" json:"type,omitempty"` +} + +func (x *Edge) Reset() { + *x = Edge{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[18] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Edge) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Edge) ProtoMessage() {} + +func (x *Edge) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[18] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Edge.ProtoReflect.Descriptor instead. +func (*Edge) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{18} +} + +func (x *Edge) GetParent() uint64 { + if x != nil { + return x.Parent + } + return 0 +} + +func (x *Edge) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Edge) GetName() []byte { + if x != nil { + return x.Name + } + return nil +} + +func (x *Edge) GetType() uint32 { + if x != nil { + return x.Type + } + return 0 +} + +type EdgeList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Edge `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *EdgeList) Reset() { + *x = EdgeList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[19] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *EdgeList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*EdgeList) ProtoMessage() {} + +func (x *EdgeList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[19] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use EdgeList.ProtoReflect.Descriptor instead. +func (*EdgeList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{19} +} + +func (x *EdgeList) GetList() []*Edge { + if x != nil { + return x.List + } + return nil +} + +// for redis and tikv only +type Parent struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Parent uint64 `protobuf:"varint,2,opt,name=parent,proto3" json:"parent,omitempty"` + Cnt int64 `protobuf:"varint,3,opt,name=cnt,proto3" json:"cnt,omitempty"` +} + +func (x *Parent) Reset() { + *x = Parent{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[20] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Parent) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Parent) ProtoMessage() {} + +func (x *Parent) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[20] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Parent.ProtoReflect.Descriptor instead. +func (*Parent) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{20} +} + +func (x *Parent) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Parent) GetParent() uint64 { + if x != nil { + return x.Parent + } + return 0 +} + +func (x *Parent) GetCnt() int64 { + if x != nil { + return x.Cnt + } + return 0 +} + +type ParentList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Parent `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *ParentList) Reset() { + *x = ParentList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[21] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ParentList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ParentList) ProtoMessage() {} + +func (x *ParentList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[21] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ParentList.ProtoReflect.Descriptor instead. +func (*ParentList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{21} +} + +func (x *ParentList) GetList() []*Parent { + if x != nil { + return x.List + } + return nil +} + +type Chunk struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Index uint32 `protobuf:"varint,2,opt,name=index,proto3" json:"index,omitempty"` + Slices []byte `protobuf:"bytes,3,opt,name=slices,proto3" json:"slices,omitempty"` // array of meta.slice +} + +func (x *Chunk) Reset() { + *x = Chunk{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[22] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Chunk) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Chunk) ProtoMessage() {} + +func (x *Chunk) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[22] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Chunk.ProtoReflect.Descriptor instead. +func (*Chunk) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{22} +} + +func (x *Chunk) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Chunk) GetIndex() uint32 { + if x != nil { + return x.Index + } + return 0 +} + +func (x *Chunk) GetSlices() []byte { + if x != nil { + return x.Slices + } + return nil +} + +type ChunkList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Chunk `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *ChunkList) Reset() { + *x = ChunkList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[23] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *ChunkList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*ChunkList) ProtoMessage() {} + +func (x *ChunkList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[23] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use ChunkList.ProtoReflect.Descriptor instead. +func (*ChunkList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{23} +} + +func (x *ChunkList) GetList() []*Chunk { + if x != nil { + return x.List + } + return nil +} + +type Symlink struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Inode uint64 `protobuf:"varint,1,opt,name=inode,proto3" json:"inode,omitempty"` + Target []byte `protobuf:"bytes,2,opt,name=target,proto3" json:"target,omitempty"` +} + +func (x *Symlink) Reset() { + *x = Symlink{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[24] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Symlink) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Symlink) ProtoMessage() {} + +func (x *Symlink) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[24] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Symlink.ProtoReflect.Descriptor instead. +func (*Symlink) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{24} +} + +func (x *Symlink) GetInode() uint64 { + if x != nil { + return x.Inode + } + return 0 +} + +func (x *Symlink) GetTarget() []byte { + if x != nil { + return x.Target + } + return nil +} + +type SymlinkList struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + List []*Symlink `protobuf:"bytes,1,rep,name=list,proto3" json:"list,omitempty"` +} + +func (x *SymlinkList) Reset() { + *x = SymlinkList{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[25] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *SymlinkList) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SymlinkList) ProtoMessage() {} + +func (x *SymlinkList) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[25] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SymlinkList.ProtoReflect.Descriptor instead. +func (*SymlinkList) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{25} +} + +func (x *SymlinkList) GetList() []*Symlink { + if x != nil { + return x.List + } + return nil +} + +type Footer struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Magic uint32 `protobuf:"varint,1,opt,name=magic,proto3" json:"magic,omitempty"` + Version uint32 `protobuf:"varint,2,opt,name=version,proto3" json:"version,omitempty"` + Infos map[string]*Footer_SegInfo `protobuf:"bytes,3,rep,name=infos,proto3" json:"infos,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` +} + +func (x *Footer) Reset() { + *x = Footer{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[26] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Footer) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Footer) ProtoMessage() {} + +func (x *Footer) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[26] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Footer.ProtoReflect.Descriptor instead. +func (*Footer) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{26} +} + +func (x *Footer) GetMagic() uint32 { + if x != nil { + return x.Magic + } + return 0 +} + +func (x *Footer) GetVersion() uint32 { + if x != nil { + return x.Version + } + return 0 +} + +func (x *Footer) GetInfos() map[string]*Footer_SegInfo { + if x != nil { + return x.Infos + } + return nil +} + +type Footer_SegInfo struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + Offset []uint64 `protobuf:"varint,1,rep,packed,name=offset,proto3" json:"offset,omitempty"` +} + +func (x *Footer_SegInfo) Reset() { + *x = Footer_SegInfo{} + mi := &file_pkg_meta_pb_backup_proto_msgTypes[27] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) +} + +func (x *Footer_SegInfo) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Footer_SegInfo) ProtoMessage() {} + +func (x *Footer_SegInfo) ProtoReflect() protoreflect.Message { + mi := &file_pkg_meta_pb_backup_proto_msgTypes[27] + if x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Footer_SegInfo.ProtoReflect.Descriptor instead. +func (*Footer_SegInfo) Descriptor() ([]byte, []int) { + return file_pkg_meta_pb_backup_proto_rawDescGZIP(), []int{26, 0} +} + +func (x *Footer_SegInfo) GetOffset() []uint64 { + if x != nil { + return x.Offset + } + return nil +} + +var File_pkg_meta_pb_backup_proto protoreflect.FileDescriptor + +var file_pkg_meta_pb_backup_proto_rawDesc = []byte{ + 0x0a, 0x18, 0x70, 0x6b, 0x67, 0x2f, 0x6d, 0x65, 0x74, 0x61, 0x2f, 0x70, 0x62, 0x2f, 0x62, 0x61, + 0x63, 0x6b, 0x75, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x02, 0x70, 0x62, 0x22, 0x1c, + 0x0a, 0x06, 0x46, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0xc4, 0x01, 0x0a, + 0x08, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x75, 0x73, 0x65, + 0x64, 0x53, 0x70, 0x61, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x75, 0x73, + 0x65, 0x64, 0x53, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x75, 0x73, 0x65, 0x64, 0x49, + 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x73, 0x65, + 0x64, 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x49, + 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x65, 0x78, 0x74, + 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x43, 0x68, 0x75, + 0x6e, 0x6b, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x43, 0x68, + 0x75, 0x6e, 0x6b, 0x12, 0x20, 0x0a, 0x0b, 0x6e, 0x65, 0x78, 0x74, 0x53, 0x65, 0x73, 0x73, 0x69, + 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0b, 0x6e, 0x65, 0x78, 0x74, 0x53, 0x65, + 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x54, 0x72, 0x61, + 0x73, 0x68, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6e, 0x65, 0x78, 0x74, 0x54, 0x72, + 0x61, 0x73, 0x68, 0x22, 0x35, 0x0a, 0x09, 0x53, 0x75, 0x73, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x64, + 0x12, 0x10, 0x0a, 0x03, 0x73, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x03, 0x73, + 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x02, 0x20, 0x03, + 0x28, 0x04, 0x52, 0x06, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x32, 0x0a, 0x0d, 0x53, 0x75, + 0x73, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x21, 0x0a, 0x04, 0x6c, + 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0d, 0x2e, 0x70, 0x62, 0x2e, 0x53, + 0x75, 0x73, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x64, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x4f, + 0x0a, 0x07, 0x44, 0x65, 0x6c, 0x46, 0x69, 0x6c, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, + 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, + 0x16, 0x0a, 0x06, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, + 0x06, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x65, 0x78, 0x70, 0x69, 0x72, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x65, 0x78, 0x70, 0x69, 0x72, 0x65, 0x22, + 0x2e, 0x0a, 0x0b, 0x44, 0x65, 0x6c, 0x46, 0x69, 0x6c, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1f, + 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, + 0x62, 0x2e, 0x44, 0x65, 0x6c, 0x46, 0x69, 0x6c, 0x65, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, + 0x42, 0x0a, 0x08, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x66, 0x12, 0x0e, 0x0a, 0x02, 0x69, + 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x73, + 0x69, 0x7a, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, + 0x12, 0x0a, 0x04, 0x72, 0x65, 0x66, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x04, 0x72, + 0x65, 0x66, 0x73, 0x22, 0x30, 0x0a, 0x0c, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x66, 0x4c, + 0x69, 0x73, 0x74, 0x12, 0x20, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x0c, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x6c, 0x69, 0x63, 0x65, 0x52, 0x65, 0x66, 0x52, + 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x29, 0x0a, 0x03, 0x41, 0x63, 0x6c, 0x12, 0x0e, 0x0a, 0x02, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x02, 0x69, 0x64, 0x12, 0x12, 0x0a, 0x04, + 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, + 0x22, 0x26, 0x0a, 0x07, 0x41, 0x63, 0x6c, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1b, 0x0a, 0x04, 0x6c, + 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x07, 0x2e, 0x70, 0x62, 0x2e, 0x41, + 0x63, 0x6c, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x47, 0x0a, 0x05, 0x58, 0x61, 0x74, 0x74, + 0x72, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, + 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, + 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, + 0x65, 0x22, 0x2a, 0x0a, 0x09, 0x58, 0x61, 0x74, 0x74, 0x72, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, + 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x09, 0x2e, 0x70, + 0x62, 0x2e, 0x58, 0x61, 0x74, 0x74, 0x72, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x95, 0x01, + 0x0a, 0x05, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x1a, 0x0a, + 0x08, 0x6d, 0x61, 0x78, 0x53, 0x70, 0x61, 0x63, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, + 0x08, 0x6d, 0x61, 0x78, 0x53, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x6d, 0x61, 0x78, + 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x6d, 0x61, + 0x78, 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x64, 0x53, + 0x70, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x75, 0x73, 0x65, 0x64, + 0x53, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1e, 0x0a, 0x0a, 0x75, 0x73, 0x65, 0x64, 0x49, 0x6e, 0x6f, + 0x64, 0x65, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0a, 0x75, 0x73, 0x65, 0x64, 0x49, + 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x2a, 0x0a, 0x09, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x4c, 0x69, + 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, + 0x32, 0x09, 0x2e, 0x70, 0x62, 0x2e, 0x51, 0x75, 0x6f, 0x74, 0x61, 0x52, 0x04, 0x6c, 0x69, 0x73, + 0x74, 0x22, 0x7a, 0x0a, 0x04, 0x53, 0x74, 0x61, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, + 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, + 0x1e, 0x0a, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x0a, 0x64, 0x61, 0x74, 0x61, 0x4c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x12, + 0x1c, 0x0a, 0x09, 0x75, 0x73, 0x65, 0x64, 0x53, 0x70, 0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x75, 0x73, 0x65, 0x64, 0x53, 0x70, 0x61, 0x63, 0x65, 0x12, 0x1e, 0x0a, + 0x0a, 0x75, 0x73, 0x65, 0x64, 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x0a, 0x75, 0x73, 0x65, 0x64, 0x49, 0x6e, 0x6f, 0x64, 0x65, 0x73, 0x22, 0x28, 0x0a, + 0x08, 0x53, 0x74, 0x61, 0x74, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x04, 0x6c, 0x69, 0x73, + 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x74, 0x61, + 0x74, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x30, 0x0a, 0x04, 0x4e, 0x6f, 0x64, 0x65, 0x12, + 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, + 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x28, 0x0a, 0x08, 0x4e, 0x6f, 0x64, + 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, + 0x03, 0x28, 0x0b, 0x32, 0x08, 0x2e, 0x70, 0x62, 0x2e, 0x4e, 0x6f, 0x64, 0x65, 0x52, 0x04, 0x6c, + 0x69, 0x73, 0x74, 0x22, 0x5c, 0x0a, 0x04, 0x45, 0x64, 0x67, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x70, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x02, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, + 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x12, 0x0a, + 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x04, 0x74, 0x79, 0x70, + 0x65, 0x22, 0x28, 0x0a, 0x08, 0x45, 0x64, 0x67, 0x65, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1c, 0x0a, + 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x08, 0x2e, 0x70, 0x62, + 0x2e, 0x45, 0x64, 0x67, 0x65, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x48, 0x0a, 0x06, 0x50, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x16, 0x0a, 0x06, 0x70, + 0x61, 0x72, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x06, 0x70, 0x61, 0x72, + 0x65, 0x6e, 0x74, 0x12, 0x10, 0x0a, 0x03, 0x63, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x03, 0x63, 0x6e, 0x74, 0x22, 0x2c, 0x0a, 0x0a, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x4c, + 0x69, 0x73, 0x74, 0x12, 0x1e, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x0a, 0x2e, 0x70, 0x62, 0x2e, 0x50, 0x61, 0x72, 0x65, 0x6e, 0x74, 0x52, 0x04, 0x6c, + 0x69, 0x73, 0x74, 0x22, 0x4b, 0x0a, 0x05, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x14, 0x0a, 0x05, + 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, + 0x64, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x18, 0x02, 0x20, 0x01, 0x28, + 0x0d, 0x52, 0x05, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x6c, 0x69, 0x63, + 0x65, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x6c, 0x69, 0x63, 0x65, 0x73, + 0x22, 0x2a, 0x0a, 0x09, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1d, 0x0a, + 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x09, 0x2e, 0x70, 0x62, + 0x2e, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x52, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0x37, 0x0a, 0x07, + 0x53, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x12, 0x14, 0x0a, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x69, 0x6e, 0x6f, 0x64, 0x65, 0x12, 0x16, 0x0a, + 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, + 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, 0x2e, 0x0a, 0x0b, 0x53, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, + 0x4c, 0x69, 0x73, 0x74, 0x12, 0x1f, 0x0a, 0x04, 0x6c, 0x69, 0x73, 0x74, 0x18, 0x01, 0x20, 0x03, + 0x28, 0x0b, 0x32, 0x0b, 0x2e, 0x70, 0x62, 0x2e, 0x53, 0x79, 0x6d, 0x6c, 0x69, 0x6e, 0x6b, 0x52, + 0x04, 0x6c, 0x69, 0x73, 0x74, 0x22, 0xd6, 0x01, 0x0a, 0x06, 0x46, 0x6f, 0x6f, 0x74, 0x65, 0x72, + 0x12, 0x14, 0x0a, 0x05, 0x6d, 0x61, 0x67, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x05, 0x6d, 0x61, 0x67, 0x69, 0x63, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x2b, 0x0a, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x15, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6f, 0x74, 0x65, 0x72, 0x2e, 0x49, 0x6e, 0x66, 0x6f, + 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x05, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x1a, 0x21, 0x0a, + 0x07, 0x53, 0x65, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, + 0x65, 0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x04, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, + 0x1a, 0x4c, 0x0a, 0x0a, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, + 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, + 0x12, 0x28, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x12, 0x2e, 0x70, 0x62, 0x2e, 0x46, 0x6f, 0x6f, 0x74, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x67, 0x49, + 0x6e, 0x66, 0x6f, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x42, 0x06, + 0x5a, 0x04, 0x2e, 0x2f, 0x70, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_pkg_meta_pb_backup_proto_rawDescOnce sync.Once + file_pkg_meta_pb_backup_proto_rawDescData = file_pkg_meta_pb_backup_proto_rawDesc +) + +func file_pkg_meta_pb_backup_proto_rawDescGZIP() []byte { + file_pkg_meta_pb_backup_proto_rawDescOnce.Do(func() { + file_pkg_meta_pb_backup_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_meta_pb_backup_proto_rawDescData) + }) + return file_pkg_meta_pb_backup_proto_rawDescData +} + +var file_pkg_meta_pb_backup_proto_msgTypes = make([]protoimpl.MessageInfo, 29) +var file_pkg_meta_pb_backup_proto_goTypes = []any{ + (*Format)(nil), // 0: pb.Format + (*Counters)(nil), // 1: pb.Counters + (*Sustained)(nil), // 2: pb.Sustained + (*SustainedList)(nil), // 3: pb.SustainedList + (*DelFile)(nil), // 4: pb.DelFile + (*DelFileList)(nil), // 5: pb.DelFileList + (*SliceRef)(nil), // 6: pb.SliceRef + (*SliceRefList)(nil), // 7: pb.SliceRefList + (*Acl)(nil), // 8: pb.Acl + (*AclList)(nil), // 9: pb.AclList + (*Xattr)(nil), // 10: pb.Xattr + (*XattrList)(nil), // 11: pb.XattrList + (*Quota)(nil), // 12: pb.Quota + (*QuotaList)(nil), // 13: pb.QuotaList + (*Stat)(nil), // 14: pb.Stat + (*StatList)(nil), // 15: pb.StatList + (*Node)(nil), // 16: pb.Node + (*NodeList)(nil), // 17: pb.NodeList + (*Edge)(nil), // 18: pb.Edge + (*EdgeList)(nil), // 19: pb.EdgeList + (*Parent)(nil), // 20: pb.Parent + (*ParentList)(nil), // 21: pb.ParentList + (*Chunk)(nil), // 22: pb.Chunk + (*ChunkList)(nil), // 23: pb.ChunkList + (*Symlink)(nil), // 24: pb.Symlink + (*SymlinkList)(nil), // 25: pb.SymlinkList + (*Footer)(nil), // 26: pb.Footer + (*Footer_SegInfo)(nil), // 27: pb.Footer.SegInfo + nil, // 28: pb.Footer.InfosEntry +} +var file_pkg_meta_pb_backup_proto_depIdxs = []int32{ + 2, // 0: pb.SustainedList.list:type_name -> pb.Sustained + 4, // 1: pb.DelFileList.list:type_name -> pb.DelFile + 6, // 2: pb.SliceRefList.list:type_name -> pb.SliceRef + 8, // 3: pb.AclList.list:type_name -> pb.Acl + 10, // 4: pb.XattrList.list:type_name -> pb.Xattr + 12, // 5: pb.QuotaList.list:type_name -> pb.Quota + 14, // 6: pb.StatList.list:type_name -> pb.Stat + 16, // 7: pb.NodeList.list:type_name -> pb.Node + 18, // 8: pb.EdgeList.list:type_name -> pb.Edge + 20, // 9: pb.ParentList.list:type_name -> pb.Parent + 22, // 10: pb.ChunkList.list:type_name -> pb.Chunk + 24, // 11: pb.SymlinkList.list:type_name -> pb.Symlink + 28, // 12: pb.Footer.infos:type_name -> pb.Footer.InfosEntry + 27, // 13: pb.Footer.InfosEntry.value:type_name -> pb.Footer.SegInfo + 14, // [14:14] is the sub-list for method output_type + 14, // [14:14] is the sub-list for method input_type + 14, // [14:14] is the sub-list for extension type_name + 14, // [14:14] is the sub-list for extension extendee + 0, // [0:14] is the sub-list for field type_name +} + +func init() { file_pkg_meta_pb_backup_proto_init() } +func file_pkg_meta_pb_backup_proto_init() { + if File_pkg_meta_pb_backup_proto != nil { + return + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_pkg_meta_pb_backup_proto_rawDesc, + NumEnums: 0, + NumMessages: 29, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_pkg_meta_pb_backup_proto_goTypes, + DependencyIndexes: file_pkg_meta_pb_backup_proto_depIdxs, + MessageInfos: file_pkg_meta_pb_backup_proto_msgTypes, + }.Build() + File_pkg_meta_pb_backup_proto = out.File + file_pkg_meta_pb_backup_proto_rawDesc = nil + file_pkg_meta_pb_backup_proto_goTypes = nil + file_pkg_meta_pb_backup_proto_depIdxs = nil +} diff --git a/pkg/meta/pb/backup.proto b/pkg/meta/pb/backup.proto new file mode 100644 index 000000000000..d26d53c4917d --- /dev/null +++ b/pkg/meta/pb/backup.proto @@ -0,0 +1,153 @@ +syntax = "proto3"; +package pb; +option go_package = "./pb"; + +/* +1. install protocol buffer compiler +2. install Go protoc plugin (protoc-gen-go) +3. exec: protoc --go_out=pkg/meta pkg/meta/pb/backup.proto in main directory +*/ + +message Format { + bytes data = 1; // meta.Format's json format +} + +message Counters { + int64 usedSpace = 1; + int64 usedInodes = 2; + int64 nextInode = 3; + int64 nextChunk = 4; + int64 nextSession = 5; + int64 nextTrash = 6; +} + +message Sustained { + uint64 sid = 1; + repeated uint64 inodes = 2; +} + +message SustainedList { + repeated Sustained list = 1; +} + +message DelFile { + uint64 inode = 1; + uint64 length = 2; + int64 expire = 3; +} + +message DelFileList { + repeated DelFile list = 1; +} + +message SliceRef { + uint64 id = 1; + uint32 size = 2; + int64 refs = 3; +} + +message SliceRefList { + repeated SliceRef list = 1; +} + +message Acl { + uint32 id = 1; + bytes data = 2; // acl.Rule's binary format +} + +message AclList { + repeated Acl list = 1; +} + +message Xattr { + uint64 inode = 1; + string name = 2; + bytes value = 3; +} + +message XattrList { + repeated Xattr list = 1; +} + +message Quota { + uint64 inode = 1; + int64 maxSpace = 2; + int64 maxInodes = 3; + int64 usedSpace = 4; + int64 usedInodes = 5; +} + +message QuotaList { + repeated Quota list = 1; +} + +message Stat { + uint64 inode = 1; + int64 dataLength = 2; + int64 usedSpace = 3; + int64 usedInodes = 4; +} + +message StatList { + repeated Stat list = 1; +} + +message Node { + uint64 inode = 1; + bytes data = 2; // meta.Attr's binary format +} + +message NodeList { + repeated Node list = 1; +} + +message Edge { + uint64 parent = 1; + uint64 inode = 2; + bytes name = 3; + uint32 type = 4; +} + +message EdgeList { + repeated Edge list = 1; +} + +// for redis and tikv only +message Parent { + uint64 inode = 1; + uint64 parent = 2 ; + int64 cnt = 3; +} + +message ParentList { + repeated Parent list = 1; +} + +message Chunk { + uint64 inode = 1; + uint32 index = 2; + bytes slices = 3; // array of meta.slice +} + +message ChunkList { + repeated Chunk list = 1; +} + +message Symlink { + uint64 inode = 1; + bytes target = 2; +} + +message SymlinkList { + repeated Symlink list = 1; +} + +message Footer { + message SegInfo { + repeated uint64 offset = 1; + } + + uint32 magic = 1; + uint32 version = 2; + map infos = 3; +} \ No newline at end of file diff --git a/pkg/meta/redis_bak.go b/pkg/meta/redis_bak.go new file mode 100644 index 000000000000..19a8fe739ebd --- /dev/null +++ b/pkg/meta/redis_bak.go @@ -0,0 +1,37 @@ +//go:build !noredis +// +build !noredis + +/* + * JuiceFS, Copyright 2024 Juicedata, 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 meta + +func (m *redisMeta) buildDumpedSeg(typ int, opt *DumpOption, txn *eTxn) iDumpedSeg { + return nil +} + +func (m *redisMeta) buildLoadedSeg(typ int, opt *LoadOption) iLoadedSeg { + return nil +} + +func (m *redisMeta) execETxn(ctx Context, txn *eTxn, f func(Context, *eTxn) error) error { + txn.opt.notUsed = true + return f(ctx, txn) +} + +func (m *redisMeta) prepareLoad(ctx Context, opt *LoadOption) error { + return nil +} diff --git a/pkg/meta/sql.go b/pkg/meta/sql.go index b01f41b29b5e..ed98d1c8743b 100644 --- a/pkg/meta/sql.go +++ b/pkg/meta/sql.go @@ -22,6 +22,7 @@ package meta import ( "bufio" "bytes" + "context" "database/sql" "encoding/json" "fmt" @@ -355,7 +356,7 @@ func (m *dbMeta) syncTable(beans ...interface{}) error { return err } -func (m *dbMeta) doInit(format *Format, force bool) error { +func (m *dbMeta) syncAllTables() error { if err := m.syncTable(new(setting), new(counter)); err != nil { return fmt.Errorf("create table setting, counter: %s", err) } @@ -383,10 +384,16 @@ func (m *dbMeta) doInit(format *Format, force bool) error { if err := m.syncTable(new(acl)); err != nil { return fmt.Errorf("create table acl: %s", err) } + return nil +} +func (m *dbMeta) doInit(format *Format, force bool) error { + if err := m.syncAllTables(); err != nil { + return err + } var s = setting{Name: "format"} var ok bool - err := m.roTxn(func(ses *xorm.Session) (err error) { + err := m.roTxn(Background, func(ses *xorm.Session) (err error) { ok, err = ses.Get(&s) return err }) @@ -475,7 +482,7 @@ func (m *dbMeta) cacheACLs(ctx Context) error { if !m.getFormat().EnableACL { return nil } - return m.roTxn(func(s *xorm.Session) error { + return m.roTxn(ctx, func(s *xorm.Session) error { return s.Table(&acl{}).Iterate(new(acl), func(idx int, bean interface{}) error { a := bean.(*acl) m.aclCache.Put(a.Id, a.toRule()) @@ -493,7 +500,7 @@ func (m *dbMeta) Reset() error { } func (m *dbMeta) doLoad() (data []byte, err error) { - err = m.roTxn(func(ses *xorm.Session) error { + err = m.roTxn(Background, func(ses *xorm.Session) error { if ok, err := ses.IsTableExist(&setting{}); err != nil { return err } else if !ok { @@ -585,7 +592,7 @@ func (m *dbMeta) getSession(row interface{}, detail bool) (*Session, error) { frows []flock prows []plock ) - err := m.roTxn(func(ses *xorm.Session) error { + err := m.roTxn(Background, func(ses *xorm.Session) error { if err := ses.Find(&srows, &sustained{Sid: s.Sid}); err != nil { return fmt.Errorf("find sustained %d: %s", s.Sid, err) } @@ -619,7 +626,7 @@ func (m *dbMeta) getSession(row interface{}, detail bool) (*Session, error) { } func (m *dbMeta) GetSession(sid uint64, detail bool) (s *Session, err error) { - err = m.roTxn(func(ses *xorm.Session) error { + err = m.roTxn(Background, func(ses *xorm.Session) error { if ok, err := ses.IsTableExist(&session2{}); err != nil { return err } else if ok { @@ -649,7 +656,7 @@ func (m *dbMeta) GetSession(sid uint64, detail bool) (s *Session, err error) { func (m *dbMeta) ListSessions() ([]*Session, error) { var sessions []*Session - err := m.roTxn(func(ses *xorm.Session) error { + err := m.roTxn(Background, func(ses *xorm.Session) error { if ok, err := ses.IsTableExist(&session2{}); err != nil { return err } else if ok { @@ -690,7 +697,7 @@ func (m *dbMeta) ListSessions() ([]*Session, error) { } func (m *dbMeta) getCounter(name string) (v int64, err error) { - err = m.roTxn(func(s *xorm.Session) error { + err = m.roTxn(Background, func(s *xorm.Session) error { c := counter{Name: name} _, err := s.Get(&c) if err == nil { @@ -831,7 +838,7 @@ func (m *dbMeta) txn(f func(s *xorm.Session) error, inodes ...Ino) error { return lastErr } -func (m *dbMeta) roTxn(f func(s *xorm.Session) error) error { +func (m *dbMeta) roTxn(ctx context.Context, f func(s *xorm.Session) error) error { start := time.Now() defer func() { m.txDist.Observe(time.Since(start).Seconds()) }() s := m.db.NewSession() @@ -842,8 +849,15 @@ func (m *dbMeta) roTxn(f func(s *xorm.Session) error) error { opt.Isolation = sql.LevelRepeatableRead } + var maxRetry int + val := ctx.Value(txMaxRetryKey{}) + if val == nil { + maxRetry = 50 + } else { + maxRetry = val.(int) + } var lastErr error - for i := 0; i < 50; i++ { + for i := 0; i < maxRetry; i++ { err := s.BeginTx(&opt) if err != nil && opt.ReadOnly && (strings.Contains(err.Error(), "READ") || strings.Contains(err.Error(), "driver does not support read-only transactions")) { logger.Warnf("the database does not support read-only transaction") @@ -954,7 +968,7 @@ func (m *dbMeta) doFlushStats() { } func (m *dbMeta) doLookup(ctx Context, parent Ino, name string, inode *Ino, attr *Attr) syscall.Errno { - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { s = s.Table(&edge{}) nn := namedNode{node: node{Parent: parent}, Name: []byte(name)} var exist bool @@ -978,7 +992,7 @@ func (m *dbMeta) doLookup(ctx Context, parent Ino, name string, inode *Ino, attr } func (m *dbMeta) doGetAttr(ctx Context, inode Ino, attr *Attr) syscall.Errno { - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { var n = node{Inode: inode} ok, err := s.Get(&n) if err != nil { @@ -1221,7 +1235,7 @@ func (m *dbMeta) doFallocate(ctx Context, inode Ino, mode uint8, off uint64, siz func (m *dbMeta) doReadlink(ctx Context, inode Ino, noatime bool) (atime int64, target []byte, err error) { if noatime { - err = m.roTxn(func(s *xorm.Session) error { + err = m.roTxn(ctx, func(s *xorm.Session) error { var l = symlink{Inode: inode} ok, err := s.Get(&l) if err == nil && ok { @@ -2099,7 +2113,7 @@ func (m *dbMeta) doLink(ctx Context, inode, parent Ino, name string, attr *Attr) } func (m *dbMeta) doReaddir(ctx Context, inode Ino, plus uint8, entries *[]*Entry, limit int) syscall.Errno { - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { s = s.Table(&edge{}) if plus != 0 { s = s.Join("INNER", &node{}, "jfs_edge.inode=jfs_node.inode") @@ -2150,7 +2164,7 @@ func (m *dbMeta) doCleanStaleSession(sid uint64) error { } var sus []sustained - err = m.roTxn(func(ses *xorm.Session) error { + err = m.roTxn(Background, func(ses *xorm.Session) error { sus = nil return ses.Find(&sus, &sustained{Sid: sid}) }) @@ -2186,7 +2200,7 @@ func (m *dbMeta) doCleanStaleSession(sid uint64) error { func (m *dbMeta) doFindStaleSessions(limit int) ([]uint64, error) { var sids []uint64 - _ = m.roTxn(func(ses *xorm.Session) error { + _ = m.roTxn(Background, func(ses *xorm.Session) error { var ss []session2 err := ses.Where("Expire < ?", time.Now().Unix()).Limit(limit, 0).Find(&ss) if err != nil { @@ -2203,7 +2217,7 @@ func (m *dbMeta) doFindStaleSessions(limit int) ([]uint64, error) { return sids, nil } - err := m.roTxn(func(ses *xorm.Session) error { + err := m.roTxn(Background, func(ses *xorm.Session) error { if ok, err := ses.IsTableExist(&session{}); err != nil { return err } else if ok { @@ -2269,7 +2283,7 @@ func (m *dbMeta) doDeleteSustainedInode(sid uint64, inode Ino) error { func (m *dbMeta) doRead(ctx Context, inode Ino, indx uint32) ([]*slice, syscall.Errno) { var c = chunk{Inode: inode, Indx: indx} - if err := m.roTxn(func(s *xorm.Session) error { + if err := m.roTxn(ctx, func(s *xorm.Session) error { _, err := s.MustCols("indx").Get(&c) return err }); err != nil { @@ -2484,7 +2498,7 @@ func (m *dbMeta) getParents(s *xorm.Session, inode, parent Ino) []Ino { func (m *dbMeta) doGetParents(ctx Context, inode Ino) map[Ino]int { var rows []edge - if err := m.roTxn(func(s *xorm.Session) error { + if err := m.roTxn(ctx, func(s *xorm.Session) error { rows = nil return s.Find(&rows, &edge{Inode: inode}) }); err != nil { @@ -2572,7 +2586,7 @@ func (m *dbMeta) doGetDirStat(ctx Context, ino Ino, trySync bool) (*dirStat, sys st := dirStats{Inode: ino} var exist bool var err error - if err = m.roTxn(func(s *xorm.Session) error { + if err = m.roTxn(ctx, func(s *xorm.Session) error { exist, err = s.Get(&st) return err }); err != nil { @@ -2611,7 +2625,7 @@ func (m *dbMeta) doGetDirStat(ctx Context, ino Ino, trySync bool) (*dirStat, sys func (m *dbMeta) doFindDeletedFiles(ts int64, limit int) (map[Ino]uint64, error) { files := make(map[Ino]uint64) - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { var ds []delfile err := s.Where("expire < ?", ts).Limit(limit, 0).Find(&ds) if err != nil { @@ -2627,7 +2641,7 @@ func (m *dbMeta) doFindDeletedFiles(ts int64, limit int) (map[Ino]uint64, error) func (m *dbMeta) doCleanupSlices() { var cks []sliceRef - _ = m.roTxn(func(s *xorm.Session) error { + _ = m.roTxn(Background, func(s *xorm.Session) error { cks = nil return s.Where("refs <= 0").Find(&cks) }) @@ -2676,7 +2690,7 @@ func (m *dbMeta) deleteChunk(inode Ino, indx uint32) error { continue } var ref = sliceRef{Id: s.id} - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { ok, err := s.Get(&ref) if err == nil && !ok { err = errors.New("not found") @@ -2692,7 +2706,7 @@ func (m *dbMeta) deleteChunk(inode Ino, indx uint32) error { func (m *dbMeta) doDeleteFileData(inode Ino, length uint64) { var indexes []chunk - _ = m.roTxn(func(s *xorm.Session) error { + _ = m.roTxn(Background, func(s *xorm.Session) error { indexes = nil return s.Cols("indx").Find(&indexes, &chunk{Inode: inode}) }) @@ -2716,7 +2730,7 @@ func (m *dbMeta) doCleanupDelayedSlices(edge int64) (int, error) { var result []delslices var batch int = 1e6 for { - _ = m.roTxn(func(s *xorm.Session) error { + _ = m.roTxn(Background, func(s *xorm.Session) error { result = result[:0] return s.Where("deleted < ?", edge).Limit(batch, 0).Find(&result) }) @@ -2747,7 +2761,7 @@ func (m *dbMeta) doCleanupDelayedSlices(edge int64) (int, error) { } for _, s := range ss { var ref = sliceRef{Id: s.Id} - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { ok, err := s.Get(&ref) if err == nil && !ok { err = errors.New("not found") @@ -2808,7 +2822,7 @@ func (m *dbMeta) doCompactChunk(inode Ino, indx uint32, origin []byte, ss []*sli // there could be false-negative that the compaction is successful, double-check if st != 0 && st != syscall.EINVAL { var ok bool - if err := m.roTxn(func(s *xorm.Session) error { + if err := m.roTxn(Background, func(s *xorm.Session) error { var e error ok, e = s.Get(&sliceRef{Id: id}) return e @@ -2833,7 +2847,7 @@ func (m *dbMeta) doCompactChunk(inode Ino, indx uint32, origin []byte, ss []*sli } var ref = sliceRef{Id: s.id} var ok bool - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { var e error ok, e = s.Get(&ref) return e @@ -2853,7 +2867,7 @@ func dup(b []byte) []byte { } func (m *dbMeta) scanAllChunks(ctx Context, ch chan<- cchunk, bar *utils.Bar) error { - return m.roTxn(func(s *xorm.Session) error { + return m.roTxn(ctx, func(s *xorm.Session) error { return s.Table(&chunk{}).Iterate(new(chunk), func(idx int, bean interface{}) error { c := bean.(*chunk) if len(c.Slices) > sliceBytes { @@ -2869,7 +2883,7 @@ func (m *dbMeta) ListSlices(ctx Context, slices map[Ino][]Slice, scanPending, de if delete { m.doCleanupSlices() } - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(ctx, func(s *xorm.Session) error { var cs []chunk err := s.Find(&cs) if err != nil { @@ -2897,7 +2911,7 @@ func (m *dbMeta) ListSlices(ctx Context, slices map[Ino][]Slice, scanPending, de } if scanPending { - _ = m.roTxn(func(s *xorm.Session) error { + _ = m.roTxn(ctx, func(s *xorm.Session) error { var cks []sliceRef err := s.Where("refs <= 0").Find(&cks) if err != nil { @@ -2930,7 +2944,7 @@ func (m *dbMeta) scanTrashSlices(ctx Context, scan trashSliceScan) error { } var dss []delslices - err := m.roTxn(func(tx *xorm.Session) error { + err := m.roTxn(ctx, func(tx *xorm.Session) error { if ok, err := tx.IsTableExist(&delslices{}); err != nil { return err } else if !ok { @@ -2975,7 +2989,7 @@ func (m *dbMeta) scanTrashSlices(ctx Context, scan trashSliceScan) error { if clean { for _, s := range ss { var ref = sliceRef{Id: s.Id} - err := m.roTxn(func(tx *xorm.Session) error { + err := m.roTxn(ctx, func(tx *xorm.Session) error { ok, err := tx.Get(&ref) if err == nil && !ok { err = errors.New("not found") @@ -2996,7 +3010,7 @@ func (m *dbMeta) scanPendingSlices(ctx Context, scan pendingSliceScan) error { return nil } var refs []sliceRef - err := m.roTxn(func(tx *xorm.Session) error { + err := m.roTxn(ctx, func(tx *xorm.Session) error { if ok, err := tx.IsTableExist(&sliceRef{}); err != nil { return err } else if !ok { @@ -3027,7 +3041,7 @@ func (m *dbMeta) scanPendingFiles(ctx Context, scan pendingFileScan) error { } var dfs []delfile - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(ctx, func(s *xorm.Session) error { if ok, err := s.IsTableExist(&delfile{}); err != nil { return err } else if !ok { @@ -3093,7 +3107,7 @@ func (m *dbMeta) doRepair(ctx Context, inode Ino, attr *Attr) syscall.Errno { func (m *dbMeta) GetXattr(ctx Context, inode Ino, name string, vbuff *[]byte) syscall.Errno { defer m.timeit("GetXattr", time.Now()) inode = m.checkRoot(inode) - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { var x = xattr{Inode: inode, Name: name} ok, err := s.Get(&x) if err != nil { @@ -3110,7 +3124,7 @@ func (m *dbMeta) GetXattr(ctx Context, inode Ino, name string, vbuff *[]byte) sy func (m *dbMeta) ListXattr(ctx Context, inode Ino, names *[]byte) syscall.Errno { defer m.timeit("ListXattr", time.Now()) inode = m.checkRoot(inode) - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { var xs []xattr err := s.Where("inode = ?", inode).Find(&xs, &xattr{Inode: inode}) if err != nil { @@ -3182,7 +3196,7 @@ func (m *dbMeta) doRemoveXattr(ctx Context, inode Ino, name string) syscall.Errn func (m *dbMeta) doGetQuota(ctx Context, inode Ino) (*Quota, error) { var quota *Quota - return quota, m.roTxn(func(s *xorm.Session) error { + return quota, m.roTxn(ctx, func(s *xorm.Session) error { q := dirQuota{Inode: inode} ok, e := s.Get(&q) if e == nil && ok { @@ -3247,7 +3261,7 @@ func (m *dbMeta) doDelQuota(ctx Context, inode Ino) error { func (m *dbMeta) doLoadQuotas(ctx Context) (map[Ino]*Quota, error) { var rows []dirQuota - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(ctx, func(s *xorm.Session) error { rows = rows[:0] return s.Find(&rows) }) @@ -3480,7 +3494,7 @@ func (m *dbMeta) dumpDir(s *xorm.Session, inode Ino, tree *DumpedEntry, bw *bufi conds[c] = sync.NewCond(&ms[c]) if c < len(entries) { go func(c int) { - _ = m.roTxn(func(s *xorm.Session) error { + _ = m.roTxn(Background, func(s *xorm.Session) error { for i := c; i < len(entries) && err == nil; i += threads { e := entries[i] er := m.dumpEntry(s, e.Attr.Inode, 0, e, showProgress) @@ -3656,7 +3670,7 @@ func (m *dbMeta) DumpMeta(w io.Writer, root Ino, threads int, keepSecret, fast, progress := utils.NewProgress(false) var tree, trash *DumpedEntry root = m.checkRoot(root) - return m.roTxn(func(s *xorm.Session) error { + return m.roTxn(Background, func(s *xorm.Session) error { if root == RootInode && fast { defer func() { m.snap = nil }() bar := progress.AddCountBar("Snapshot keys", 0) @@ -3880,7 +3894,20 @@ func (m *dbMeta) loadEntry(e *DumpedEntry, chs []chan interface{}, aclMaxId *uin chs[0] <- n } -func (m *dbMeta) LoadMeta(r io.Reader) error { +func (m *dbMeta) getTxnBatchNum() int { + switch m.Name() { + case "sqlite3": + return 999 / MaxFieldsCountOfTable + case "mysql": + return 65535 / MaxFieldsCountOfTable + case "postgres": + return 1000 + default: + return 1000 + } +} + +func (m *dbMeta) checkAddr() error { tables, err := m.db.DBMetas() if err != nil { return err @@ -3890,41 +3917,20 @@ func (m *dbMeta) LoadMeta(r io.Reader) error { if !strings.Contains(addr, "://") { addr = fmt.Sprintf("%s://%s", m.Name(), addr) } - return fmt.Errorf("Database %s is not empty", addr) - } - if err = m.syncTable(new(setting), new(counter)); err != nil { - return fmt.Errorf("create table setting, counter: %s", err) - } - if err = m.syncTable(new(node), new(edge), new(symlink), new(xattr)); err != nil { - return fmt.Errorf("create table node, edge, symlink, xattr: %s", err) - } - if err = m.syncTable(new(chunk), new(sliceRef), new(delslices)); err != nil { - return fmt.Errorf("create table chunk, chunk_ref, delslices: %s", err) + return fmt.Errorf("database %s is not empty", addr) } - if err = m.syncTable(new(session2), new(sustained), new(delfile)); err != nil { - return fmt.Errorf("create table session2, sustaind, delfile: %s", err) - } - if err = m.syncTable(new(flock), new(plock), new(dirQuota)); err != nil { - return fmt.Errorf("create table flock, plock, dirQuota: %s", err) - } - if err := m.syncTable(new(dirStats)); err != nil { - return fmt.Errorf("create table dirStats: %s", err) - } - if err := m.syncTable(new(detachedNode)); err != nil { - return fmt.Errorf("create table detachedNode: %s", err) - } - if err = m.syncTable(new(acl)); err != nil { - return fmt.Errorf("create table acl: %s", err) + return nil +} + +func (m *dbMeta) LoadMeta(r io.Reader) error { + if err := m.checkAddr(); err != nil { + return err } - var batch int - switch m.Name() { - case "sqlite3": - batch = 999 / MaxFieldsCountOfTable - case "mysql": - batch = 65535 / MaxFieldsCountOfTable - case "postgres": - batch = 1000 + if err := m.syncAllTables(); err != nil { + return err } + + batch := m.getTxnBatchNum() chs := make([]chan interface{}, 6) // node, edge, chunk, chunkRef, xattr, others insert := func(index int, beans []interface{}) error { return m.txn(func(s *xorm.Session) error { @@ -4176,7 +4182,7 @@ func (m *dbMeta) doCloneEntry(ctx Context, srcIno Ino, parent Ino, name string, func (m *dbMeta) doFindDetachedNodes(t time.Time) []Ino { var inodes []Ino - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { var nodes []detachedNode err := s.Where("added < ?", t.Unix()).Find(&nodes) for _, n := range nodes { @@ -4424,7 +4430,7 @@ func (m *dbMeta) doSetFacl(ctx Context, ino Ino, aclType uint8, rule *aclAPI.Rul } func (m *dbMeta) doGetFacl(ctx Context, ino Ino, aclType uint8, aclId uint32, rule *aclAPI.Rule) syscall.Errno { - return errno(m.roTxn(func(s *xorm.Session) error { + return errno(m.roTxn(ctx, func(s *xorm.Session) error { if aclId == aclAPI.None { attr := &Attr{} n := &node{Inode: ino} @@ -4496,7 +4502,7 @@ func (m *dbMeta) newDirHandler(inode Ino, plus bool, entries []*Entry) DirHandle func (m *dbMeta) getDirFetcher() dirFetcher { return func(ctx Context, inode Ino, cursor interface{}, offset, limit int, plus bool) (interface{}, []*Entry, error) { entries := make([]*Entry, 0, limit) - err := m.roTxn(func(s *xorm.Session) error { + err := m.roTxn(Background, func(s *xorm.Session) error { var name []byte if cursor != nil { name = cursor.([]byte) diff --git a/pkg/meta/sql_bak.go b/pkg/meta/sql_bak.go new file mode 100644 index 000000000000..c887694aabc7 --- /dev/null +++ b/pkg/meta/sql_bak.go @@ -0,0 +1,960 @@ +//go:build !nosqlite || !nomysql || !nopg +// +build !nosqlite !nomysql !nopg + +/* + * JuiceFS, Copyright 2024 Juicedata, 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 meta + +import ( + "context" + "fmt" + "sync" + "sync/atomic" + "time" + + aclAPI "github.com/juicedata/juicefs/pkg/acl" + "github.com/juicedata/juicefs/pkg/meta/pb" + "github.com/juicedata/juicefs/pkg/utils" + "golang.org/x/sync/errgroup" + "google.golang.org/protobuf/proto" + "xorm.io/xorm" +) + +var ( + sqlDumpBatchSize = 40960 +) + +func (m *dbMeta) buildDumpedSeg(typ int, opt *DumpOption, txn *eTxn) iDumpedSeg { + ds := dumpedSeg{typ: typ, meta: m, opt: opt, txn: txn} + switch typ { + case SegTypeFormat: + return &formatDS{ds} + case SegTypeCounter: + return &sqlCounterDS{ds} + case SegTypeSustained: + return &sqlSustainedDS{ds} + case SegTypeDelFile: + return &sqlDelFileDS{ds} + case SegTypeSliceRef: + return &sqlSliceRefDS{dumpedBatchSeg{ds, []*sync.Pool{{New: func() interface{} { return &pb.SliceRef{} }}}}} + case SegTypeAcl: + return &sqlAclDS{ds} + case SegTypeXattr: + return &sqlXattrDS{ds} + case SegTypeQuota: + return &sqlQuotaDS{ds} + case SegTypeStat: + return &sqlStatDS{ds} + case SegTypeNode: + return &sqlNodeDBS{dumpedBatchSeg{ds, []*sync.Pool{{New: func() interface{} { return &pb.Node{} }}}}} + case SegTypeChunk: + return &sqlChunkDBS{dumpedBatchSeg{ds, []*sync.Pool{{New: func() interface{} { return &pb.Chunk{} }}}}} + case SegTypeEdge: + return &sqlEdgeDBS{dumpedBatchSeg{ds, []*sync.Pool{{New: func() interface{} { return &pb.Edge{} }}}}, sync.Mutex{}} + case SegTypeParent: + return &sqlParentDS{ds} + case SegTypeSymlink: + return &sqlSymlinkDBS{dumpedBatchSeg{ds, []*sync.Pool{{New: func() interface{} { return &pb.Symlink{} }}}}} + } + return nil +} + +var sqlLoadedPoolOnce sync.Once +var sqlLoadedPools = make(map[int][]*sync.Pool) + +func (m *dbMeta) buildLoadedPools(typ int) []*sync.Pool { + sqlLoadedPoolOnce.Do(func() { + sqlLoadedPools = map[int][]*sync.Pool{ + SegTypeNode: {{New: func() interface{} { return &node{} }}}, + SegTypeChunk: {{New: func() interface{} { return &chunk{} }}}, + SegTypeEdge: {{New: func() interface{} { return &edge{} }}}, + SegTypeSymlink: {{New: func() interface{} { return &symlink{} }}}, + } + }) + return sqlLoadedPools[typ] +} + +func (m *dbMeta) buildLoadedSeg(typ int, opt *LoadOption) iLoadedSeg { + switch typ { + case SegTypeFormat: + return &sqlFormatLS{loadedSeg{typ: typ, meta: m}} + case SegTypeCounter: + return &sqlCounterLS{loadedSeg{typ: typ, meta: m}} + case SegTypeSustained: + return &sqlSustainedLS{loadedSeg{typ: typ, meta: m}} + case SegTypeDelFile: + return &sqlDelFileLS{loadedSeg{typ: typ, meta: m}} + case SegTypeSliceRef: + return &sqlSliceRefLS{loadedSeg{typ: typ, meta: m}} + case SegTypeAcl: + return &sqlAclLS{loadedSeg{typ: typ, meta: m}} + case SegTypeXattr: + return &sqlXattrLS{loadedSeg{typ: typ, meta: m}} + case SegTypeQuota: + return &sqlQuotaLS{loadedSeg{typ: typ, meta: m}} + case SegTypeStat: + return &sqlStatLS{loadedSeg{typ: typ, meta: m}} + case SegTypeNode: + return &sqlNodeLS{loadedSeg{typ: typ, meta: m}, m.buildLoadedPools(typ)} + case SegTypeChunk: + return &sqlChunkLS{loadedSeg{typ: typ, meta: m}, m.buildLoadedPools(typ)} + case SegTypeEdge: + return &sqlEdgeLS{loadedSeg{typ: typ, meta: m}, m.buildLoadedPools(typ)} + case SegTypeParent: + return &sqlParentLS{loadedSeg{typ: typ, meta: m}} + case SegTypeSymlink: + return &sqlSymlinkLS{loadedSeg{typ: typ, meta: m}, m.buildLoadedPools(typ)} + } + return nil +} + +func (m *dbMeta) execETxn(ctx Context, txn *eTxn, f func(Context, *eTxn) error) error { + if txn.opt.coNum > 1 { + // only use same txn when coNum == 1 for sql + txn.opt.notUsed = true + return f(ctx, txn) + } + ctx.WithValue(txMaxRetryKey{}, txn.opt.maxRetry) + return m.roTxn(ctx, func(sess *xorm.Session) error { + txn.obj = sess + return f(ctx, txn) + }) +} + +func (m *dbMeta) execStmt(ctx context.Context, txn *eTxn, f func(*xorm.Session) error) error { + if txn.opt.notUsed { + return m.roTxn(ctx, func(s *xorm.Session) error { + return f(s) + }) + } + + var err error + cnt := 0 + for cnt < txn.opt.maxStmtRetry { + err = f(txn.obj.(*xorm.Session)) + if err == nil || !m.shouldRetry(err) { + break + } + cnt++ + time.Sleep(time.Duration(cnt) * time.Microsecond) + } + return err +} + +func getSQLCounterFields(c *pb.Counters) map[string]*int64 { + return map[string]*int64{ + usedSpace: &c.UsedSpace, + totalInodes: &c.UsedInodes, + "nextInode": &c.NextInode, + "nextChunk": &c.NextChunk, + "nextSession": &c.NextSession, + "nextTrash": &c.NextTrash, + } +} + +type sqlCounterDS struct { + dumpedSeg +} + +func (s *sqlCounterDS) dump(ctx Context, ch chan *dumpedResult) error { + meta := s.meta.(*dbMeta) + var rows []counter + if err := meta.execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + counters := &pb.Counters{} + fieldMap := getSQLCounterFields(counters) + for _, row := range rows { + if fieldPtr, ok := fieldMap[row.Name]; ok { + *fieldPtr = row.Value + } + } + if err := dumpResult(ctx, ch, &dumpedResult{s, counters}); err != nil { + return err + } + logger.Debugf("dump %s result %+v", s, counters) + return nil +} + +type sqlSustainedDS struct { + dumpedSeg +} + +func (s *sqlSustainedDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []sustained + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + ss := make(map[uint64][]uint64) + for _, row := range rows { + ss[row.Sid] = append(ss[row.Sid], uint64(row.Inode)) + } + + pss := &pb.SustainedList{ + List: make([]*pb.Sustained, 0, len(ss)), + } + for k, v := range ss { + pss.List = append(pss.List, &pb.Sustained{Sid: k, Inodes: v}) + } + + if err := dumpResult(ctx, ch, &dumpedResult{s, pss}); err != nil { + return err + } + logger.Debugf("dump %s num %d", s, len(ss)) + return nil +} + +type sqlDelFileDS struct { + dumpedSeg +} + +func (s *sqlDelFileDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []delfile + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + delFiles := &pb.DelFileList{List: make([]*pb.DelFile, 0, len(rows))} + for _, row := range rows { + delFiles.List = append(delFiles.List, &pb.DelFile{Inode: uint64(row.Inode), Length: row.Length, Expire: row.Expire}) + } + if err := dumpResult(ctx, ch, &dumpedResult{s, delFiles}); err != nil { + return err + } + logger.Debugf("dump %s num %d", s, len(delFiles.List)) + return nil +} + +type sqlSliceRefDS struct { + dumpedBatchSeg +} + +func (s *sqlSliceRefDS) dump(ctx Context, ch chan *dumpedResult) error { + eg, _ := errgroup.WithContext(ctx) + eg.SetLimit(s.opt.CoNum) + + taskFinished := false + psrs := &pb.SliceRefList{List: make([]*pb.SliceRef, 0, 1024)} + for start := 0; !taskFinished; start += sqlDumpBatchSize { + nStart := start + eg.Go(func() error { + var rows []sliceRef + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Where("refs != 1").Limit(sqlDumpBatchSize, nStart).Find(&rows) // skip default refs + }); err != nil || len(rows) == 0 { + taskFinished = true + return err + } + var psr *pb.SliceRef + for _, sr := range rows { + psr = s.pools[0].Get().(*pb.SliceRef) + psr.Id = sr.Id + psr.Size = sr.Size + psr.Refs = int64(sr.Refs) + psrs.List = append(psrs.List, psr) + } + return nil + }) + } + if err := eg.Wait(); err != nil { + logger.Errorf("query %s err: %v", s, err) + return err + } + if err := dumpResult(ctx, ch, &dumpedResult{s, psrs}); err != nil { + return err + } + logger.Debugf("dump %s num %d", s, len(psrs.List)) + return nil +} + +func (s *sqlSliceRefDS) release(msg proto.Message) { + psrs := msg.(*pb.SliceRefList) + for _, psr := range psrs.List { + s.pools[0].Put(psr) + } + psrs.List = nil +} + +type sqlAclDS struct { + dumpedSeg +} + +func (s *sqlAclDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []acl + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + acls := &pb.AclList{List: make([]*pb.Acl, 0, len(rows))} + for _, row := range rows { + acls.List = append(acls.List, &pb.Acl{ + Id: row.Id, + Data: row.toRule().Encode(), + }) + } + if err := dumpResult(ctx, ch, &dumpedResult{s, acls}); err != nil { + return err + } + logger.Debugf("dump %s num %d", s, len(acls.List)) + return nil +} + +type sqlXattrDS struct { + dumpedSeg +} + +func (s *sqlXattrDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []xattr + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + + if len(rows) == 0 { + return nil + } + + pxs := &pb.XattrList{ + List: make([]*pb.Xattr, 0, len(rows)), + } + for _, x := range rows { + pxs.List = append(pxs.List, &pb.Xattr{ + Inode: uint64(x.Inode), + Name: x.Name, + Value: x.Value, + }) + } + + logger.Debugf("dump %s num %d", s, len(pxs.List)) + return dumpResult(ctx, ch, &dumpedResult{s, pxs}) +} + +type sqlQuotaDS struct { + dumpedSeg +} + +func (s *sqlQuotaDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []dirQuota + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + if len(rows) == 0 { + return nil + } + pqs := &pb.QuotaList{ + List: make([]*pb.Quota, 0, len(rows)), + } + for _, q := range rows { + pqs.List = append(pqs.List, &pb.Quota{ + Inode: uint64(q.Inode), + MaxSpace: q.MaxSpace, + MaxInodes: q.MaxInodes, + UsedSpace: q.UsedSpace, + UsedInodes: q.UsedInodes, + }) + } + logger.Debugf("dump %s num %d", s, len(pqs.List)) + return dumpResult(ctx, ch, &dumpedResult{s, pqs}) +} + +type sqlStatDS struct { + dumpedSeg +} + +func (s *sqlStatDS) dump(ctx Context, ch chan *dumpedResult) error { + var rows []dirStats + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Find(&rows) + }); err != nil { + return err + } + if len(rows) == 0 { + return nil + } + pss := &pb.StatList{ + List: make([]*pb.Stat, 0, len(rows)), + } + for _, st := range rows { + pss.List = append(pss.List, &pb.Stat{ + Inode: uint64(st.Inode), + DataLength: st.DataLength, + UsedInodes: st.UsedInodes, + UsedSpace: st.UsedSpace, + }) + } + logger.Debugf("dump %s num %d", s, len(pss.List)) + return dumpResult(ctx, ch, &dumpedResult{s, pss}) +} + +func sqlQueryBatch(ctx Context, s iDumpedSeg, opt *DumpOption, ch chan *dumpedResult, query func(ctx context.Context, limit, start int, sum *int64) (proto.Message, error)) error { + eg, egCtx := errgroup.WithContext(ctx) + eg.SetLimit(opt.CoNum) + + taskFinished := false + sum := int64(0) + for start := 0; !taskFinished; start += sqlDumpBatchSize { + nStart := start + eg.Go(func() error { + msg, err := query(egCtx, sqlDumpBatchSize, nStart, &sum) + if err != nil || msg == nil { + taskFinished = true + return err + } + return dumpResult(egCtx, ch, &dumpedResult{s, msg}) + }) + } + if err := eg.Wait(); err != nil { + logger.Errorf("query %s err: %v", s, err) + return err + } + logger.Debugf("dump %s num %d", s, sum) + return nil +} + +type sqlNodeDBS struct { + dumpedBatchSeg +} + +func (s *sqlNodeDBS) dump(ctx Context, ch chan *dumpedResult) error { + return sqlQueryBatch(ctx, s, s.opt, ch, s.doQuery) +} + +func (s *sqlNodeDBS) doQuery(ctx context.Context, limit, start int, sum *int64) (proto.Message, error) { + var rows []node + m := s.meta.(*dbMeta) + if err := m.execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Limit(limit, start).Find(&rows) + }); err != nil { + return nil, err + } + if len(rows) == 0 { + return nil, nil + } + pns := &pb.NodeList{ + List: make([]*pb.Node, 0, len(rows)), + } + var pn *pb.Node + attr := &Attr{} + for _, n := range rows { + pn = s.pools[0].Get().(*pb.Node) + pn.Inode = uint64(n.Inode) + m.parseAttr(&n, attr) + pn.Data = m.marshal(attr) + pns.List = append(pns.List, pn) + } + atomic.AddInt64(sum, int64(len(pns.List))) + return pns, nil +} + +func (s *sqlNodeDBS) release(msg proto.Message) { + pns := msg.(*pb.NodeList) + for _, node := range pns.List { + s.pools[0].Put(node) + } + pns.List = nil +} + +type sqlChunkDBS struct { + dumpedBatchSeg +} + +func (s *sqlChunkDBS) dump(ctx Context, ch chan *dumpedResult) error { + return sqlQueryBatch(ctx, s, s.opt, ch, s.doQuery) +} + +func (s *sqlChunkDBS) doQuery(ctx context.Context, limit, start int, sum *int64) (proto.Message, error) { + var rows []chunk + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Limit(limit, start).Find(&rows) + }); err != nil { + return nil, err + } + if len(rows) == 0 { + return nil, nil + } + pcs := &pb.ChunkList{ + List: make([]*pb.Chunk, 0, len(rows)), + } + var pc *pb.Chunk + for _, c := range rows { + pc = s.pools[0].Get().(*pb.Chunk) + pc.Inode = uint64(c.Inode) + pc.Index = c.Indx + pc.Slices = c.Slices + pcs.List = append(pcs.List, pc) + } + atomic.AddInt64(sum, int64(len(pcs.List))) + return pcs, nil +} + +func (s *sqlChunkDBS) release(msg proto.Message) { + pcs := msg.(*pb.ChunkList) + for _, pc := range pcs.List { + s.pools[0].Put(pc) + } + pcs.List = nil +} + +type sqlEdgeDBS struct { + dumpedBatchSeg + lock sync.Mutex +} + +func (s *sqlEdgeDBS) dump(ctx Context, ch chan *dumpedResult) error { + ctx.WithValue("parents", make(map[uint64][]uint64)) + return sqlQueryBatch(ctx, s, s.opt, ch, s.doQuery) +} + +func (s *sqlEdgeDBS) doQuery(ctx context.Context, limit, start int, sum *int64) (proto.Message, error) { + // TODO: optimize parents + s.lock.Lock() + parents := ctx.Value("parents").(map[uint64][]uint64) + s.lock.Unlock() + + var rows []edge + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Limit(limit, start).Find(&rows) + }); err != nil { + return nil, err + } + if len(rows) == 0 { + return nil, nil + } + pes := &pb.EdgeList{ + List: make([]*pb.Edge, 0, len(rows)), + } + var pe *pb.Edge + for _, e := range rows { + pe = s.pools[0].Get().(*pb.Edge) + pe.Parent = uint64(e.Parent) + pe.Inode = uint64(e.Inode) + pe.Name = e.Name + pe.Type = uint32(e.Type) + + s.lock.Lock() + parents[uint64(e.Inode)] = append(parents[uint64(e.Inode)], uint64(e.Parent)) + s.lock.Unlock() + pes.List = append(pes.List, pe) + } + atomic.AddInt64(sum, int64(len(pes.List))) + return pes, nil +} + +func (s *sqlEdgeDBS) release(msg proto.Message) { + pes := msg.(*pb.EdgeList) + for _, pe := range pes.List { + s.pools[0].Put(pe) + } + pes.List = nil +} + +type sqlParentDS struct { + dumpedSeg +} + +func (s *sqlParentDS) dump(ctx Context, ch chan *dumpedResult) error { + val := ctx.Value("parents") + if val == nil { + return nil + } + + parents := val.(map[uint64][]uint64) + pls := &pb.ParentList{ + List: make([]*pb.Parent, 0, sqlDumpBatchSize), + } + st := make(map[uint64]int64) + for inode, ps := range parents { + if len(ps) > 1 { + for k := range st { + delete(st, k) + } + for _, p := range ps { + st[p] = st[p] + 1 + } + for parent, cnt := range st { + pls.List = append(pls.List, &pb.Parent{Inode: inode, Parent: parent, Cnt: cnt}) + } + } + if len(pls.List) >= sqlDumpBatchSize { + if err := dumpResult(ctx, ch, &dumpedResult{s, pls}); err != nil { + return err + } + pls = &pb.ParentList{ + List: make([]*pb.Parent, 0, sqlDumpBatchSize), + } + } + } + + if len(pls.List) > 0 { + if err := dumpResult(ctx, ch, &dumpedResult{s, pls}); err != nil { + return err + } + } + return nil +} + +type sqlSymlinkDBS struct { + dumpedBatchSeg +} + +func (s *sqlSymlinkDBS) dump(ctx Context, ch chan *dumpedResult) error { + return sqlQueryBatch(ctx, s, s.opt, ch, s.doQuery) +} + +func (s *sqlSymlinkDBS) doQuery(ctx context.Context, limit, start int, sum *int64) (proto.Message, error) { + var rows []symlink + if err := s.meta.(*dbMeta).execStmt(ctx, s.txn, func(s *xorm.Session) error { + rows = rows[:0] + return s.Limit(limit, start).Find(&rows) + }); err != nil { + return nil, err + } + if len(rows) == 0 { + return nil, nil + } + pss := &pb.SymlinkList{ + List: make([]*pb.Symlink, 0, len(rows)), + } + var ps *pb.Symlink + for _, sl := range rows { + ps = s.pools[0].Get().(*pb.Symlink) + ps.Inode = uint64(sl.Inode) + ps.Target = sl.Target + pss.List = append(pss.List, ps) + } + atomic.AddInt64(sum, int64(len(pss.List))) + return pss, nil +} + +func (s *sqlSymlinkDBS) release(msg proto.Message) { + pss := msg.(*pb.SymlinkList) + for _, ps := range pss.List { + s.pools[0].Put(ps) + } + pss.List = nil +} + +type sqlFormatLS struct { + loadedSeg +} + +func (s *sqlFormatLS) load(ctx Context, msg proto.Message) error { + return s.meta.(*dbMeta).insertRows([]interface{}{ + &setting{ + Name: "format", + Value: string(msg.(*pb.Format).Data), + }, + }) +} + +type sqlCounterLS struct { + loadedSeg +} + +func (s *sqlCounterLS) load(ctx Context, msg proto.Message) error { + counters := msg.(*pb.Counters) + fields := getSQLCounterFields(counters) + + var rows []interface{} + for name, field := range fields { + rows = append(rows, counter{Name: name, Value: *field}) + } + logger.Debugf("insert counters %+v", rows) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlSustainedLS struct { + loadedSeg +} + +func (s *sqlSustainedLS) load(ctx Context, msg proto.Message) error { + sustaineds := msg.(*pb.SustainedList) + rows := make([]interface{}, 0, len(sustaineds.List)) + for _, s := range sustaineds.List { + for _, inode := range s.Inodes { + rows = append(rows, sustained{Sid: s.Sid, Inode: Ino(inode)}) + } + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlDelFileLS struct { + loadedSeg +} + +func (s *sqlDelFileLS) load(ctx Context, msg proto.Message) error { + delfiles := msg.(*pb.DelFileList) + rows := make([]interface{}, 0, len(delfiles.List)) + for _, f := range delfiles.List { + rows = append(rows, &delfile{Inode: Ino(f.Inode), Length: f.Length, Expire: f.Expire}) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlSliceRefLS struct { + loadedSeg +} + +func (s *sqlSliceRefLS) load(ctx Context, msg proto.Message) error { + srs := msg.(*pb.SliceRefList) + rows := make([]interface{}, 0, len(srs.List)) + for _, sr := range srs.List { + rows = append(rows, &sliceRef{Id: sr.Id, Size: sr.Size, Refs: int(sr.Refs)}) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlAclLS struct { + loadedSeg +} + +func (s *sqlAclLS) load(ctx Context, msg proto.Message) error { + acls := msg.(*pb.AclList) + rows := make([]interface{}, 0, len(acls.List)) + for _, pa := range acls.List { + rule := &aclAPI.Rule{} + rule.Decode(pa.Data) + acl := newSQLAcl(rule) + acl.Id = pa.Id + rows = append(rows, acl) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlXattrLS struct { + loadedSeg +} + +func (s *sqlXattrLS) load(ctx Context, msg proto.Message) error { + xattrs := msg.(*pb.XattrList) + rows := make([]interface{}, 0, len(xattrs.List)) + for _, x := range xattrs.List { + rows = append(rows, &xattr{Inode: Ino(x.Inode), Name: x.Name, Value: x.Value}) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlQuotaLS struct { + loadedSeg +} + +func (s *sqlQuotaLS) load(ctx Context, msg proto.Message) error { + quotas := msg.(*pb.QuotaList) + rows := make([]interface{}, 0, len(quotas.List)) + for _, q := range quotas.List { + rows = append(rows, &dirQuota{ + Inode: Ino(q.Inode), + MaxSpace: q.MaxSpace, + MaxInodes: q.MaxInodes, + UsedSpace: q.UsedSpace, + UsedInodes: q.UsedInodes, + }) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlStatLS struct { + loadedSeg +} + +func (s *sqlStatLS) load(ctx Context, msg proto.Message) error { + stats := msg.(*pb.StatList) + rows := make([]interface{}, 0, len(stats.List)) + for _, st := range stats.List { + rows = append(rows, &dirStats{ + Inode: Ino(st.Inode), + DataLength: st.DataLength, + UsedInodes: st.UsedInodes, + UsedSpace: st.UsedSpace, + }) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return s.meta.(*dbMeta).insertRows(rows) +} + +type sqlNodeLS struct { + loadedSeg + pools []*sync.Pool +} + +func (s *sqlNodeLS) load(ctx Context, msg proto.Message) error { + nodes := msg.(*pb.NodeList) + m := s.meta.(*dbMeta) + b := m.getBase() + rows := make([]interface{}, 0, len(nodes.List)) + var pn *node + attr := &Attr{} + for _, n := range nodes.List { + pn = s.pools[0].Get().(*node) + pn.Inode = Ino(n.Inode) + attr.Parent, attr.AccessACL, attr.DefaultACL = 0, 0, 0 + b.parseAttr(n.Data, attr) + m.parseNode(attr, pn) + rows = append(rows, pn) + } + err := s.meta.(*dbMeta).insertRows(rows) + for _, n := range rows { + s.pools[0].Put(n) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return err +} + +type sqlChunkLS struct { + loadedSeg + pools []*sync.Pool +} + +func (s *sqlChunkLS) load(ctx Context, msg proto.Message) error { + chunks := msg.(*pb.ChunkList) + rows := make([]interface{}, 0, len(chunks.List)) + var pc *chunk + for _, c := range chunks.List { + pc = s.pools[0].Get().(*chunk) + pc.Id = 0 + pc.Inode = Ino(c.Inode) + pc.Indx = c.Index + pc.Slices = c.Slices + rows = append(rows, pc) + } + err := s.meta.(*dbMeta).insertRows(rows) + + for _, chk := range rows { + s.pools[0].Put(chk) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return err +} + +type sqlEdgeLS struct { + loadedSeg + pools []*sync.Pool +} + +func (s *sqlEdgeLS) load(ctx Context, msg proto.Message) error { + edges := msg.(*pb.EdgeList) + rows := make([]interface{}, 0, len(edges.List)) + var pe *edge + for _, e := range edges.List { + pe = s.pools[0].Get().(*edge) + pe.Id = 0 + pe.Parent = Ino(e.Parent) + pe.Inode = Ino(e.Inode) + pe.Name = e.Name + pe.Type = uint8(e.Type) + rows = append(rows, pe) + } + + err := s.meta.(*dbMeta).insertRows(rows) + for _, e := range rows { + s.pools[0].Put(e) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return err +} + +type sqlParentLS struct { + loadedSeg +} + +func (s *sqlParentLS) load(ctx Context, msg proto.Message) error { + return nil // No need for SQL, skip. +} + +type sqlSymlinkLS struct { + loadedSeg + pools []*sync.Pool +} + +func (s *sqlSymlinkLS) load(ctx Context, msg proto.Message) error { + symlinks := msg.(*pb.SymlinkList) + rows := make([]interface{}, 0, len(symlinks.List)) + var ps *symlink + for _, sl := range symlinks.List { + ps = s.pools[0].Get().(*symlink) + ps.Inode = Ino(sl.Inode) + ps.Target = sl.Target + rows = append(rows, ps) + } + + err := s.meta.(*dbMeta).insertRows(rows) + for _, sl := range rows { + s.pools[0].Put(sl) + } + logger.Debugf("insert %s num %d", s, len(rows)) + return err +} + +func (m *dbMeta) insertRows(beans []interface{}) error { + insert := func(rows []interface{}) error { + return m.txn(func(s *xorm.Session) error { + n, err := s.Insert(rows...) + if err == nil && int(n) != len(rows) { + err = fmt.Errorf("only %d records inserted", n) + } + return err + }) + } + + batch := m.getTxnBatchNum() + for len(beans) > 0 { + bs := utils.Min(batch, len(beans)) + if err := insert(beans[:bs]); err != nil { + logger.Errorf("Write %d beans: %s", bs, err) + return err + } + beans = beans[bs:] + } + return nil +} + +func (m *dbMeta) prepareLoad(ctx Context, opt *LoadOption) error { + opt.check() + if err := m.checkAddr(); err != nil { + return err + } + if err := m.syncAllTables(); err != nil { + return err + } + return nil +} diff --git a/pkg/meta/tkv.go b/pkg/meta/tkv.go index f874a6a23f51..d074e8fa2cd5 100644 --- a/pkg/meta/tkv.go +++ b/pkg/meta/tkv.go @@ -19,6 +19,7 @@ package meta import ( "bufio" "bytes" + "context" "encoding/binary" "encoding/json" "fmt" @@ -801,6 +802,10 @@ func (m *kvMeta) shouldRetry(err error) bool { return m.client.shouldRetry(err) } +func (m *kvMeta) roTxn(ctx context.Context, f func(*kvTxn) error) error { + return nil +} + func (m *kvMeta) txn(f func(tx *kvTxn) error, inodes ...Ino) error { if m.conf.ReadOnly { return syscall.EROFS diff --git a/pkg/meta/tkv_bak.go b/pkg/meta/tkv_bak.go new file mode 100644 index 000000000000..847b299f03f8 --- /dev/null +++ b/pkg/meta/tkv_bak.go @@ -0,0 +1,37 @@ +/* + * JuiceFS, Copyright 2024 Juicedata, 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 meta + +func (m *kvMeta) buildDumpedSeg(typ int, opt *DumpOption, txn *eTxn) iDumpedSeg { + return nil +} + +func (m *kvMeta) buildLoadedSeg(typ int, opt *LoadOption) iLoadedSeg { + return nil +} + +func (m *kvMeta) execETxn(ctx Context, txn *eTxn, f func(Context, *eTxn) error) error { + ctx.WithValue(txMaxRetryKey{}, txn.opt.maxRetry) + return m.roTxn(ctx, func(tx *kvTxn) error { + txn.obj = tx + return f(ctx, txn) + }) +} + +func (m *kvMeta) prepareLoad(ctx Context, opt *LoadOption) error { + return nil +} diff --git a/pkg/utils/utils.go b/pkg/utils/utils.go index 8ee716bb3525..c80f52aaeb27 100644 --- a/pkg/utils/utils.go +++ b/pkg/utils/utils.go @@ -41,6 +41,13 @@ func Min(a, b int) int { return b } +func Min64(a, b uint64) uint64 { + if a < b { + return a + } + return b +} + // Exists checks if the file/folder in given path exists func Exists(path string) bool { _, err := os.Stat(path)