github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/db/checkpoint/entry.go (about)

     1  // Copyright 2021 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package checkpoint
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/container/batch"
    24  
    25  	"github.com/matrixorigin/matrixone/pkg/container/types"
    26  	"github.com/matrixorigin/matrixone/pkg/logutil"
    27  	"github.com/matrixorigin/matrixone/pkg/objectio"
    28  	"github.com/matrixorigin/matrixone/pkg/pb/api"
    29  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio"
    30  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    31  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail"
    32  )
    33  
    34  type CheckpointEntry struct {
    35  	sync.RWMutex
    36  	start, end types.TS
    37  	state      State
    38  	entryType  EntryType
    39  	cnLocation objectio.Location
    40  	tnLocation objectio.Location
    41  	lastPrint  time.Time
    42  	waterLine  time.Duration
    43  	version    uint32
    44  
    45  	ckpLSN      uint64
    46  	truncateLSN uint64
    47  }
    48  
    49  func NewCheckpointEntry(start, end types.TS, typ EntryType) *CheckpointEntry {
    50  	return &CheckpointEntry{
    51  		start:     start,
    52  		end:       end,
    53  		state:     ST_Pending,
    54  		entryType: typ,
    55  		lastPrint: time.Now(),
    56  		waterLine: time.Minute * 4,
    57  		version:   logtail.CheckpointCurrentVersion,
    58  	}
    59  }
    60  
    61  func (e *CheckpointEntry) SetVersion(version uint32) {
    62  	e.Lock()
    63  	defer e.Unlock()
    64  	e.version = version
    65  }
    66  
    67  func (e *CheckpointEntry) IncrWaterLine() {
    68  	e.Lock()
    69  	defer e.Unlock()
    70  	e.waterLine += time.Minute * 4
    71  }
    72  func (e *CheckpointEntry) SetLSN(ckpLSN, truncateLSN uint64) {
    73  	e.ckpLSN = ckpLSN
    74  	e.truncateLSN = truncateLSN
    75  }
    76  func (e *CheckpointEntry) CheckPrintTime() bool {
    77  	e.RLock()
    78  	defer e.RUnlock()
    79  	return time.Since(e.lastPrint) > e.waterLine
    80  }
    81  func (e *CheckpointEntry) LSNString() string {
    82  	if e.version < logtail.CheckpointVersion7 {
    83  		return fmt.Sprintf("version too small: v%d", e.version)
    84  	}
    85  	return fmt.Sprintf("ckp %d, truncate %d", e.ckpLSN, e.truncateLSN)
    86  }
    87  
    88  func (e *CheckpointEntry) GetStart() types.TS { return e.start }
    89  func (e *CheckpointEntry) GetEnd() types.TS   { return e.end }
    90  func (e *CheckpointEntry) GetState() State {
    91  	e.RLock()
    92  	defer e.RUnlock()
    93  	return e.state
    94  }
    95  func (e *CheckpointEntry) IsCommitted() bool {
    96  	e.RLock()
    97  	defer e.RUnlock()
    98  	return e.state == ST_Finished
    99  }
   100  func (e *CheckpointEntry) HasOverlap(from, to types.TS) bool {
   101  	if e.start.Greater(&to) || e.end.Less(&from) {
   102  		return false
   103  	}
   104  	return true
   105  }
   106  func (e *CheckpointEntry) LessEq(ts types.TS) bool {
   107  	return e.end.LessEq(&ts)
   108  }
   109  func (e *CheckpointEntry) SetLocation(cn, tn objectio.Location) {
   110  	e.Lock()
   111  	defer e.Unlock()
   112  	e.cnLocation = cn
   113  	e.tnLocation = tn
   114  }
   115  
   116  func (e *CheckpointEntry) GetLocation() objectio.Location {
   117  	e.RLock()
   118  	defer e.RUnlock()
   119  	return e.cnLocation
   120  }
   121  
   122  func (e *CheckpointEntry) GetTNLocation() objectio.Location {
   123  	e.RLock()
   124  	defer e.RUnlock()
   125  	return e.tnLocation
   126  }
   127  
   128  func (e *CheckpointEntry) GetVersion() uint32 {
   129  	return e.version
   130  }
   131  
   132  func (e *CheckpointEntry) SetState(state State) (ok bool) {
   133  	e.Lock()
   134  	defer e.Unlock()
   135  	// entry is already finished
   136  	if e.state == ST_Finished {
   137  		return
   138  	}
   139  	// entry is already running
   140  	if state == ST_Running && e.state == ST_Running {
   141  		return
   142  	}
   143  	e.state = state
   144  	ok = true
   145  	return
   146  }
   147  
   148  func (e *CheckpointEntry) IsRunning() bool {
   149  	e.RLock()
   150  	defer e.RUnlock()
   151  	return e.state == ST_Running
   152  }
   153  func (e *CheckpointEntry) IsPendding() bool {
   154  	e.RLock()
   155  	defer e.RUnlock()
   156  	return e.state == ST_Pending
   157  }
   158  func (e *CheckpointEntry) IsFinished() bool {
   159  	e.RLock()
   160  	defer e.RUnlock()
   161  	return e.state == ST_Finished
   162  }
   163  
   164  func (e *CheckpointEntry) IsIncremental() bool {
   165  	return e.entryType == ET_Incremental
   166  }
   167  
   168  func (e *CheckpointEntry) GetType() EntryType {
   169  	return e.entryType
   170  }
   171  
   172  func (e *CheckpointEntry) String() string {
   173  	t := "I"
   174  	if !e.IsIncremental() {
   175  		t = "G"
   176  	}
   177  	state := e.GetState()
   178  	return fmt.Sprintf("CKP[%s][%v][%s](%s->%s)", t, state, e.LSNString(), e.start.ToString(), e.end.ToString())
   179  }
   180  
   181  func (e *CheckpointEntry) Prefetch(
   182  	ctx context.Context,
   183  	fs *objectio.ObjectFS,
   184  	data *logtail.CheckpointData,
   185  ) (err error) {
   186  	if err = data.PrefetchFrom(
   187  		ctx,
   188  		e.version,
   189  		fs.Service,
   190  		e.tnLocation,
   191  	); err != nil {
   192  		return
   193  	}
   194  	return
   195  }
   196  
   197  func (e *CheckpointEntry) Read(
   198  	ctx context.Context,
   199  	fs *objectio.ObjectFS,
   200  	data *logtail.CheckpointData,
   201  ) (err error) {
   202  	reader, err := blockio.NewObjectReader(fs.Service, e.tnLocation)
   203  	if err != nil {
   204  		return
   205  	}
   206  
   207  	if err = data.ReadFrom(
   208  		ctx,
   209  		e.version,
   210  		e.tnLocation,
   211  		reader,
   212  		fs.Service,
   213  	); err != nil {
   214  		return
   215  	}
   216  	return
   217  }
   218  
   219  func (e *CheckpointEntry) PrefetchMetaIdx(
   220  	ctx context.Context,
   221  	fs *objectio.ObjectFS,
   222  ) (data *logtail.CheckpointData, err error) {
   223  	data = logtail.NewCheckpointData(common.CheckpointAllocator)
   224  	if err = data.PrefetchMeta(
   225  		ctx,
   226  		e.version,
   227  		fs.Service,
   228  		e.tnLocation,
   229  	); err != nil {
   230  		return
   231  	}
   232  	return
   233  }
   234  
   235  func (e *CheckpointEntry) ReadMetaIdx(
   236  	ctx context.Context,
   237  	fs *objectio.ObjectFS,
   238  	data *logtail.CheckpointData,
   239  ) (err error) {
   240  	reader, err := blockio.NewObjectReader(fs.Service, e.tnLocation)
   241  	if err != nil {
   242  		return
   243  	}
   244  	return data.ReadTNMetaBatch(ctx, e.version, e.tnLocation, reader)
   245  }
   246  
   247  func (e *CheckpointEntry) GetByTableID(ctx context.Context, fs *objectio.ObjectFS, tid uint64) (ins, del, cnIns, segDel *api.Batch, err error) {
   248  	reader, err := blockio.NewObjectReader(fs.Service, e.cnLocation)
   249  	if err != nil {
   250  		return
   251  	}
   252  	data := logtail.NewCNCheckpointData()
   253  	err = blockio.PrefetchMeta(fs.Service, e.cnLocation)
   254  	if err != nil {
   255  		return
   256  	}
   257  
   258  	err = data.PrefetchMetaIdx(ctx, e.version, logtail.GetMetaIdxesByVersion(e.version), e.cnLocation, fs.Service)
   259  	if err != nil {
   260  		return
   261  	}
   262  	err = data.InitMetaIdx(ctx, e.version, reader, e.cnLocation, common.CheckpointAllocator)
   263  	if err != nil {
   264  		return
   265  	}
   266  	err = data.PrefetchMetaFrom(ctx, e.version, e.cnLocation, fs.Service, tid)
   267  	if err != nil {
   268  		return
   269  	}
   270  	err = data.PrefetchFrom(ctx, e.version, fs.Service, e.cnLocation, tid)
   271  	if err != nil {
   272  		return
   273  	}
   274  	var bats []*batch.Batch
   275  	if bats, err = data.ReadFromData(ctx, tid, e.cnLocation, reader, e.version, common.CheckpointAllocator); err != nil {
   276  		return
   277  	}
   278  	ins, del, cnIns, segDel, err = data.GetTableDataFromBats(tid, bats)
   279  	return
   280  }
   281  
   282  func (e *CheckpointEntry) GCMetadata(fs *objectio.ObjectFS) error {
   283  	name := blockio.EncodeCheckpointMetadataFileName(CheckpointDir, PrefixMetadata, e.start, e.end)
   284  	err := fs.Delete(name)
   285  	logutil.Infof("GC checkpoint metadata %v, err %v", e.String(), err)
   286  	return err
   287  }
   288  
   289  func (e *CheckpointEntry) GCEntry(fs *objectio.ObjectFS) error {
   290  	err := fs.Delete(e.cnLocation.Name().String())
   291  	defer logutil.Infof("GC checkpoint entry %v, err %v", e.String(), err)
   292  	return err
   293  }
   294  
   295  type MetaFile struct {
   296  	index int
   297  	start types.TS
   298  	end   types.TS
   299  	name  string
   300  }
   301  
   302  func (m *MetaFile) String() string {
   303  	return fmt.Sprintf("MetaFile[%d][%s->%s][%s]", m.index, m.start.ToString(), m.end.ToString(), m.name)
   304  }
   305  
   306  func (m *MetaFile) GetIndex() int {
   307  	return m.index
   308  }
   309  
   310  func (m *MetaFile) GetStart() types.TS {
   311  	return m.start
   312  }
   313  
   314  func (m *MetaFile) GetEnd() types.TS {
   315  	return m.end
   316  }
   317  
   318  func (m *MetaFile) GetName() string {
   319  	return m.name
   320  }