Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions br/cmd/br/cmd.go
Original file line number Diff line number Diff line change
Expand Up @@ -130,6 +130,16 @@ func calculateMemoryLimit(memleft uint64) uint64 {
// * f(4GB) = 256MB
// * f(+inf) -> 512MB
memreserved := halfGiB / (1 + fourGiB/(memleft|1))

// Prevent uint64 underflow when memreserved >= memleft
// This can happen when available memory is very low (< 256MB)
if memreserved >= memleft {
log.Warn("insufficient memory left for BR, using minimum limit",
zap.Uint64("memleft", memleft),
zap.Uint64("memreserved", memreserved))
return quarterGiB // Return minimum limit (256 MB)
}

// 0 memused memtotal-memreserved memtotal
// +--------+--------------------+----------------+
// ^ br mem upper limit
Expand Down
17 changes: 13 additions & 4 deletions br/pkg/storage/s3.go
Original file line number Diff line number Diff line change
Expand Up @@ -953,10 +953,19 @@ func (rs *S3Storage) open(
return nil, RangeInfo{}, errors.Annotatef(berrors.ErrStorageUnknown, "open file '%s' failed. The S3 object has no content length", path)
}
objectSize := *(result.ContentLength)
r = RangeInfo{
Start: 0,
End: objectSize - 1,
Size: objectSize,
// Handle empty objects (size=0) to avoid End=-1
if objectSize == 0 {
r = RangeInfo{
Start: 0,
End: 0,
Size: 0,
}
} else {
r = RangeInfo{
Start: 0,
End: objectSize - 1,
Size: objectSize,
}
}
} else {
r, err = ParseRangeInfo(result.ContentRange)
Expand Down
10 changes: 10 additions & 0 deletions br/pkg/storage/storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -304,7 +304,17 @@ func ReadDataInRange(
start int64,
p []byte,
) (n int, err error) {
// Sanity check: reject obviously invalid offsets
if start < 0 {
return 0, errors.Annotatef(berrors.ErrInvalidArgument,
"invalid negative start offset: %d", start)
}
end := start + int64(len(p))
// Detect overflow: if end wrapped around to negative, overflow occurred
if end < start {
return 0, errors.Annotatef(berrors.ErrInvalidArgument,
"range calculation overflow: start=%d, len=%d", start, len(p))
}
rd, err := storage.Open(ctx, name, &ReaderOption{
StartOffset: &start,
EndOffset: &end,
Expand Down
14 changes: 13 additions & 1 deletion br/pkg/stream/decode_kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package stream

import (
"encoding/binary"
"math"

"github.com/pingcap/errors"
berrors "github.com/pingcap/tidb/br/pkg/errors"
Expand Down Expand Up @@ -63,7 +64,18 @@ func (ei *EventIterator) Next() {

// Valid checks whether the iterator is valid.
func (ei *EventIterator) Valid() bool {
return ei.err == nil && ei.pos < uint32(len(ei.buff))
if ei.err != nil {
return false
}
buffLen := len(ei.buff)
// Check if buffer length exceeds uint32 range
// This prevents truncation when comparing with ei.pos (uint32)
if buffLen > math.MaxUint32 {
ei.err = errors.Annotatef(berrors.ErrInvalidArgument,
"buffer too large: %d bytes exceeds uint32 limit (%d bytes)", buffLen, math.MaxUint32)
return false
}
return ei.pos < uint32(buffLen)
}

// Key gets the key in kv-event if valid() == true
Expand Down
16 changes: 12 additions & 4 deletions br/pkg/stream/meta_kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,14 +173,22 @@ l_for:
for len(data) > 0 {
switch data[0] {
case flagShortValuePrefix:
// Need at least 2 bytes: flag + vlen
if len(data) < 2 {
return errors.Annotatef(berrors.ErrInvalidArgument,
"insufficient data for short value prefix, need at least 2 bytes but only have %d",
len(data))
}
vlen := data[1]
if len(data) < int(vlen)+2 {
// Need: flag (1 byte) + vlen (1 byte) + value (vlen bytes)
requiredLen := int(vlen) + 2
if len(data) < requiredLen {
return errors.Annotatef(berrors.ErrInvalidArgument,
"insufficient data for short value, need %d bytes but only have %d",
int(vlen)+2, len(data))
requiredLen, len(data))
}
v.shortValue = data[2 : int(vlen)+2]
data = data[int(vlen)+2:]
v.shortValue = data[2:requiredLen]
data = data[requiredLen:]
case flagOverlappedRollback:
v.hasOverlappedRollback = true
data = data[1:]
Expand Down
12 changes: 12 additions & 0 deletions br/pkg/stream/meta_kv_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,18 @@ func TestWriteCFValueShortValueOverflow(t *testing.T) {
require.True(t, v4.HasShortValue())
require.Equal(t, len(v4.GetShortValue()), 255)
require.True(t, bytes.Equal(v4.GetShortValue(), largeValue))

// Test case 5: Edge case with only 1 byte (flag), no vlen byte - should error
buff5 := make([]byte, 0, 9)
buff5 = append(buff5, WriteTypePut)
buff5 = codec.EncodeUvarint(buff5, ts)
buff5 = append(buff5, flagShortValuePrefix) // Only flag, no vlen byte following

v5 := new(RawWriteCFValue)
err5 := v5.ParseFrom(buff5)
require.Error(t, err5)
require.Contains(t, err5.Error(), "insufficient data for short value prefix")
require.Contains(t, err5.Error(), "need at least 2 bytes but only have 1")
}

func TestWriteCFValueWithRollback(t *testing.T) {
Expand Down
9 changes: 8 additions & 1 deletion br/pkg/stream/stream_metas.go
Original file line number Diff line number Diff line change
Expand Up @@ -1173,7 +1173,14 @@ func (m MigrationExt) applyMetaEditTo(ctx context.Context, medit *pb.MetaEdit, m
medit.DeleteLogicalFiles[idx].Spans,
dfi.RangeOffset,
func(s *pb.Span, u uint64) int {
return int(s.Offset - u)
// Use comparison instead of subtraction to avoid uint64 underflow
// and int overflow issues
if s.Offset < u {
return -1
} else if s.Offset > u {
return 1
}
return 0
})
if ok && medit.DeleteLogicalFiles[idx].Spans[received].Length != dfi.RangeLength {
err = errors.Annotatef(
Expand Down
13 changes: 7 additions & 6 deletions br/pkg/task/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/meta/model"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/oracle"
)

func TestParseTSString(t *testing.T) {
Expand All @@ -32,8 +33,8 @@ func TestParseTSString(t *testing.T) {
ts, err = ParseTSString("2021-01-01 01:42:23", false)
require.NoError(t, err)
localTime := time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, time.Local)
localTimestamp := localTime.Unix()
localTSO := uint64((localTimestamp << 18) * 1000)
// Use oracle.GoTimeToTS instead of manual calculation to avoid overflow
localTSO := oracle.GoTimeToTS(localTime)
require.Equal(t, localTSO, ts)

_, err = ParseTSString("2021-01-01 01:42:23", true)
Expand All @@ -43,17 +44,17 @@ func TestParseTSString(t *testing.T) {
ts, err = ParseTSString("2021-01-01 01:42:23+00:00", true)
require.NoError(t, err)
localTime = time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, time.UTC)
localTimestamp = localTime.Unix()
localTSO = uint64((localTimestamp << 18) * 1000)
// Use oracle.GoTimeToTS instead of manual calculation to avoid overflow
localTSO = oracle.GoTimeToTS(localTime)
require.Equal(t, localTSO, ts)

ts, err = ParseTSString("2021-01-01 01:42:23+08:00", true)
require.NoError(t, err)
secondsEastOfUTC := int((8 * time.Hour).Seconds())
beijing := time.FixedZone("Beijing Time", secondsEastOfUTC)
localTime = time.Date(2021, time.Month(1), 1, 1, 42, 23, 0, beijing)
localTimestamp = localTime.Unix()
localTSO = uint64((localTimestamp << 18) * 1000)
// Use oracle.GoTimeToTS instead of manual calculation to avoid overflow
localTSO = oracle.GoTimeToTS(localTime)
require.Equal(t, localTSO, ts)
}

Expand Down
8 changes: 8 additions & 0 deletions br/pkg/task/common.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@ import (
"crypto/tls"
"encoding/hex"
"fmt"
"math"
"net/url"
"os"
"path"
Expand Down Expand Up @@ -631,6 +632,13 @@ func (cfg *Config) ParseFromFlags(flags *pflag.FlagSet) error {
if rateLimitUnit, err = flags.GetUint64(flagRateLimitUnit); err != nil {
return errors.Trace(err)
}
// Check for multiplication overflow when both values are non-zero
// This prevents silent wraparound that would cause incorrect rate limiting
if rateLimit > 0 && rateLimitUnit > 0 && rateLimit > math.MaxUint64/rateLimitUnit {
return errors.Annotatef(berrors.ErrInvalidArgument,
"rate limit calculation overflow: %d * %d exceeds uint64 max (consider max ~17PB/s)",
rateLimit, rateLimitUnit)
}
cfg.RateLimit = rateLimit * rateLimitUnit

cfg.Schemas = make(map[string]struct{})
Expand Down
Loading