Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

lightning: add function getChunkCompressedSizeForParquet for solving parquet oom issue #49021

Merged
merged 11 commits into from
Dec 11, 2023
1 change: 1 addition & 0 deletions br/pkg/lightning/importer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,7 @@ go_test(
"table_import_test.go",
"tidb_test.go",
],
data = glob(["testdata/**"]),
embed = [":importer"],
flaky = True,
shard_count = 50,
Expand Down
40 changes: 40 additions & 0 deletions br/pkg/lightning/importer/table_import.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
verify "github.com/pingcap/tidb/br/pkg/lightning/verification"
"github.com/pingcap/tidb/br/pkg/lightning/web"
"github.com/pingcap/tidb/br/pkg/lightning/worker"
"github.com/pingcap/tidb/br/pkg/storage"
"github.com/pingcap/tidb/br/pkg/version"
"github.com/pingcap/tidb/pkg/errno"
tidbkv "github.com/pingcap/tidb/pkg/kv"
Expand Down Expand Up @@ -768,6 +769,12 @@ ChunkLoop:
break
}

if chunk.FileMeta.Type == mydump.SourceTypeParquet {
// TODO: use the compressed size of the chunk to conduct memory control
_, err = getChunkCompressedSizeForParquet(ctx, chunk, rc.store)
return nil, errors.Trace(err)
}

restoreWorker := rc.regionWorkers.Apply()
wg.Add(1)
go func(w *worker.Worker, cr *chunkProcessor) {
Expand Down Expand Up @@ -1172,6 +1179,39 @@ func (tr *TableImporter) postProcess(
return true, nil
}

func getChunkCompressedSizeForParquet(
ctx context.Context,
chunk *checkpoints.ChunkCheckpoint,
store storage.ExternalStorage,
) (int64, error) {
reader, err := mydump.OpenReader(ctx, &chunk.FileMeta, store, storage.DecompressConfig{})
if err != nil {
return 0, errors.Trace(err)
}
parser, err := mydump.NewParquetParser(ctx, store, reader, chunk.FileMeta.Path)
if err != nil {
return 0, errors.Trace(err)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

close reader when failed to new parser?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

modified

}
//nolint: errcheck
defer parser.Close()
err = parser.Reader.ReadFooter()
if err != nil {
return 0, errors.Trace(err)
}
rowGroups := parser.Reader.Footer.GetRowGroups()
var maxRowGroupSize int64
for _, rowGroup := range rowGroups {
var rowGroupSize int64
columnChunks := rowGroup.GetColumns()
for _, columnChunk := range columnChunks {
columnChunkSize := columnChunk.MetaData.GetTotalCompressedSize()
rowGroupSize += columnChunkSize
}
maxRowGroupSize = max(maxRowGroupSize, rowGroupSize)
}
return maxRowGroupSize, nil
}

func updateStatsMeta(ctx context.Context, db *sql.DB, tableID int64, count int) {
s := common.SQLWithRetry{
DB: db,
Expand Down
36 changes: 36 additions & 0 deletions br/pkg/lightning/importer/table_import_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2421,3 +2421,39 @@ func TestGetDDLStatus(t *testing.T) {
require.Equal(t, model.JobStateRunning, status.state)
require.Equal(t, int64(123)+int64(456), status.rowCount)
}

func TestGetChunkCompressedSizeForParquet(t *testing.T) {
dir := "./testdata/"
fileName := "000000_0.parquet"
store, err := storage.NewLocalStorage(dir)
require.NoError(t, err)

dataFiles := make([]mydump.FileInfo, 0)
dataFiles = append(dataFiles, mydump.FileInfo{
TableName: filter.Table{Schema: "db", Name: "table"},
FileMeta: mydump.SourceFileMeta{
Path: fileName,
Type: mydump.SourceTypeParquet,
Compression: mydump.CompressionNone,
SortKey: "99",
FileSize: 192,
},
})

chunk := checkpoints.ChunkCheckpoint{
Key: checkpoints.ChunkCheckpointKey{Path: dataFiles[0].FileMeta.Path, Offset: 0},
FileMeta: dataFiles[0].FileMeta,
Chunk: mydump.Chunk{
Offset: 0,
EndOffset: 192,
PrevRowIDMax: 0,
RowIDMax: 100,
},
}
ctx, cancel := context.WithCancel(context.Background())
defer cancel()

compressedSize, err := getChunkCompressedSizeForParquet(ctx, &chunk, store)
require.NoError(t, err)
require.Equal(t, compressedSize, int64(192))
}
Binary file added br/pkg/lightning/importer/testdata/000000_0.parquet
Binary file not shown.
Loading