Skip to content

Commit 9679a3f

Browse files
authored
Lightning: Make sure we are using default block size of 16KB if user does not specify one. (#60097) (#60185)
close #59947
1 parent 7b51a54 commit 9679a3f

File tree

4 files changed

+49
-1
lines changed

4 files changed

+49
-1
lines changed

lightning/pkg/importer/table_import.go

+3
Original file line numberDiff line numberDiff line change
@@ -653,6 +653,9 @@ func (tr *TableImporter) preprocessEngine(
653653
logTask := tr.logger.With(zap.Int32("engineNumber", engineID)).Begin(zap.InfoLevel, "encode kv data and write")
654654
dataEngineCfg := &backend.EngineConfig{
655655
TableInfo: tr.tableInfo,
656+
Local: backend.LocalEngineConfig{
657+
BlockSize: int(rc.cfg.TikvImporter.BlockSize),
658+
},
656659
}
657660
if !tr.tableMeta.IsRowOrdered {
658661
dataEngineCfg.Local.Compact = true

pkg/lightning/backend/local/engine.go

+10
Original file line numberDiff line numberDiff line change
@@ -1417,6 +1417,16 @@ func newSSTWriter(path string, blockSize int) (*sstable.Writer, error) {
14171417
if err != nil {
14181418
return nil, errors.Trace(err)
14191419
}
1420+
1421+
// Logic to ensure the default block size is set to 16KB.
1422+
// If a smaller block size is used (e.g., 4KB, the default for Pebble),
1423+
// a single large SST file may generate a disproportionately large index block,
1424+
// potentially causing a memory spike and leading to an Out of Memory (OOM) scenario.
1425+
// If the user specifies a smaller block size, respect their choice.
1426+
if blockSize <= 0 {
1427+
blockSize = config.DefaultBlockSize
1428+
}
1429+
14201430
writable := objstorageprovider.NewFileWritable(f)
14211431
writer := sstable.NewWriter(writable, sstable.WriterOptions{
14221432
TablePropertyCollectors: []func() pebble.TablePropertyCollector{

pkg/lightning/backend/local/engine_test.go

+35
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"os"
2222
"path"
2323
"path/filepath"
24+
"reflect"
2425
"sync"
2526
"testing"
2627
"time"
@@ -33,6 +34,7 @@ import (
3334
"github.com/pingcap/tidb/br/pkg/membuf"
3435
"github.com/pingcap/tidb/pkg/lightning/backend"
3536
"github.com/pingcap/tidb/pkg/lightning/common"
37+
"github.com/pingcap/tidb/pkg/lightning/config"
3638
"github.com/pingcap/tidb/pkg/lightning/log"
3739
"github.com/stretchr/testify/require"
3840
"github.com/tikv/client-go/v2/oracle"
@@ -240,3 +242,36 @@ func TestIterOutputHasUniqueMemorySpace(t *testing.T) {
240242
// after iter closed, the memory buffer of iter goes to pool
241243
require.Greater(t, pool.TotalSize(), int64(0))
242244
}
245+
246+
// TestCreateSSTWriterDefaultBlockSize tests that createSSTWriter will use the default block size of 16KB if the block size is not set.
247+
func TestCreateSSTWriterDefaultBlockSize(t *testing.T) {
248+
db, tmpPath := makePebbleDB(t, nil)
249+
t.Cleanup(func() {
250+
require.NoError(t, db.Close())
251+
})
252+
engine := &Engine{
253+
config: backend.LocalEngineConfig{
254+
BlockSize: 0, // BlockSize is not set
255+
},
256+
sstDir: tmpPath,
257+
logger: log.Logger{},
258+
}
259+
260+
writer := &Writer{
261+
engine: engine,
262+
}
263+
264+
sstWriter, err := writer.createSSTWriter()
265+
require.NoError(t, err)
266+
require.NotNil(t, sstWriter)
267+
268+
// blockSize is a private field of sstWriter.writer, so we use reflection to access the private field blockSize
269+
writerValue := reflect.ValueOf(sstWriter.writer).Elem()
270+
blockSizeField := writerValue.FieldByName("blockSize")
271+
require.True(t, blockSizeField.IsValid(), "blockSize field should be valid")
272+
require.Equal(t, config.DefaultBlockSize, int(blockSizeField.Int()))
273+
274+
// clean up
275+
err = sstWriter.writer.Close()
276+
require.NoError(t, err)
277+
}

pkg/lightning/config/config.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -1484,7 +1484,7 @@ func NewConfig() *Config {
14841484
DiskQuota: ByteSize(math.MaxInt64),
14851485
DuplicateResolution: NoneOnDup,
14861486
PausePDSchedulerScope: PausePDSchedulerScopeTable,
1487-
BlockSize: 16 * 1024,
1487+
BlockSize: DefaultBlockSize,
14881488
LogicalImportBatchSize: ByteSize(defaultLogicalImportBatchSize),
14891489
LogicalImportBatchRows: defaultLogicalImportBatchRows,
14901490
},

0 commit comments

Comments
 (0)