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: a way to estimate parquet file size #46984

Merged
merged 10 commits into from
Sep 22, 2023
61 changes: 59 additions & 2 deletions br/pkg/lightning/mydump/loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,11 @@ import (
)

// sampleCompressedFileSize represents how many bytes need to be sampled for compressed files
const sampleCompressedFileSize = 4 * 1024
const (
sampleCompressedFileSize = 4 * 1024
maxSampleParquetDataSize = 8 * 1024
maxSampleParquetRowCount = 500
)

// MDDatabaseMeta contains some parsed metadata for a database in the source by MyDumper Loader.
type MDDatabaseMeta struct {
Expand Down Expand Up @@ -473,7 +477,7 @@ func (s *mdLoaderSetup) constructFileInfo(ctx context.Context, path string, size
s.tableSchemas = append(s.tableSchemas, info)
case SourceTypeViewSchema:
s.viewSchemas = append(s.viewSchemas, info)
case SourceTypeSQL, SourceTypeCSV, SourceTypeParquet:
case SourceTypeSQL, SourceTypeCSV:
if info.FileMeta.Compression != CompressionNone {
compressRatio, err2 := SampleFileCompressRatio(ctx, info.FileMeta, s.loader.GetStore())
if err2 != nil {
Expand All @@ -484,6 +488,15 @@ func (s *mdLoaderSetup) constructFileInfo(ctx context.Context, path string, size
}
}
s.tableDatas = append(s.tableDatas, info)
case SourceTypeParquet:
parquestDataSize, err2 := SampleParquetDataSize(ctx, info.FileMeta, s.loader.GetStore())
if err2 != nil {
logger.Error("fail to sample parquet data size", zap.String("category", "loader"),
zap.String("schema", res.Schema), zap.String("table", res.Name), zap.Stringer("type", res.Type), zap.Error(err2))
} else {
info.FileMeta.RealSize = parquestDataSize
}
s.tableDatas = append(s.tableDatas, info)
}

logger.Debug("file route result", zap.String("schema", res.Schema),
Expand Down Expand Up @@ -765,3 +778,47 @@ func SampleFileCompressRatio(ctx context.Context, fileMeta SourceFileMeta, store
}
return float64(tot) / float64(pos), nil
}

// SampleParquetDataSize samples the data size of the parquet file.
func SampleParquetDataSize(ctx context.Context, fileMeta SourceFileMeta, store storage.ExternalStorage) (int64, error) {
totalRowCount, err := ReadParquetFileRowCountByFile(ctx, store, fileMeta)
if err != nil {
return 0, err
}

reader, err := store.Open(ctx, fileMeta.Path)
if err != nil {
return 0, err
}
parser, err := NewParquetParser(ctx, store, reader, fileMeta.Path)
if err != nil {
//nolint: errcheck
reader.Close()
return 0, err
}
//nolint: errcheck
defer parser.Close()

var (
rowSize int64
rowCount int64
)
for {
err = parser.ReadRow()
if err != nil {
if errors.Cause(err) == io.EOF {
break
}
return 0, err
}
lastRow := parser.LastRow()
rowCount += 1
rowSize += int64(lastRow.Length)
parser.RecycleRow(lastRow)
if rowSize > maxSampleParquetDataSize || rowCount > maxSampleParquetRowCount {
break
}
}
size := int64(float64(totalRowCount) / float64(rowCount) * float64(rowSize))
return size, nil
}
48 changes: 48 additions & 0 deletions br/pkg/lightning/mydump/loader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,8 @@ import (
router "github.com/pingcap/tidb/util/table-router"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/xitongsys/parquet-go/parquet"
"github.com/xitongsys/parquet-go/writer"
)

type testMydumpLoaderSuite struct {
Expand Down Expand Up @@ -1103,3 +1105,49 @@ func TestSampleFileCompressRatio(t *testing.T) {
require.NoError(t, err)
require.InDelta(t, ratio, 5000.0/float64(bf.Len()), 1e-5)
}

func TestSampleParquetDataSize(t *testing.T) {
s := newTestMydumpLoaderSuite(t)
store, err := storage.NewLocalStorage(s.sourceDir)
require.NoError(t, err)

type row struct {
ID int64 `parquet:"name=id, type=INT64"`
Name string `parquet:"name=name, type=BYTE_ARRAY, encoding=PLAIN_DICTIONARY"`
}

ctx, cancel := context.WithCancel(context.Background())
defer cancel()

byteArray := make([]byte, 0, 40*1024)
bf := bytes.NewBuffer(byteArray)
pwriter, err := writer.NewParquetWriterFromWriter(bf, new(row), 4)
require.NoError(t, err)
pwriter.RowGroupSize = 128 * 1024 * 1024 //128M
pwriter.PageSize = 8 * 1024 //8K
pwriter.CompressionType = parquet.CompressionCodec_SNAPPY
for i := 0; i < 2000; i++ {
row := row{
ID: int64(i),
Name: fmt.Sprintf("row_name_%04d", i),
zeminzhou marked this conversation as resolved.
Show resolved Hide resolved
}
err = pwriter.Write(row)
require.NoError(t, err)
}
err = pwriter.WriteStop()
require.NoError(t, err)

fileName := "test_1.t1.parquet"
err = store.WriteFile(ctx, fileName, bf.Bytes())
require.NoError(t, err)

size, err := md.SampleParquetDataSize(ctx, md.SourceFileMeta{
Path: fileName,
}, store)
require.NoError(t, err)
//
// 42000 = 2000 rows * 21 bytes per row
// expected error within 10%, so delta = 42000 * 0.1 = 4200
//
require.InDelta(t, 42000, size, 4200)
}
Loading