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

sink(ticdc): Add csv decoder into codec #7352

Merged
merged 9 commits into from
Oct 18, 2022
Merged
Show file tree
Hide file tree
Changes from 4 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
99 changes: 99 additions & 0 deletions cdc/sink/codec/csv/csv_decoder.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
// Copyright 2022 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package csv

import (
"context"

"github.com/pingcap/errors"
lconfig "github.com/pingcap/tidb/br/pkg/lightning/config"
"github.com/pingcap/tidb/br/pkg/lightning/mydump"
"github.com/pingcap/tidb/br/pkg/lightning/worker"
"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/cdc/sink/codec"
"github.com/pingcap/tiflow/pkg/config"
cerror "github.com/pingcap/tiflow/pkg/errors"
)

type batchDecoder struct {
csvConfig *config.CSVConfig
parser *mydump.CSVParser
data []byte
msg *csvMessage
closed bool
}

// NewBatchDecoder creates a new BatchDecoder
func NewBatchDecoder(ctx context.Context, csvConfig *config.CSVConfig, value []byte) (codec.EventBatchDecoder, error) {
var backslashEscape bool

if len(csvConfig.Quote) == 0 {
backslashEscape = true
}
cfg := &lconfig.CSVConfig{
Separator: csvConfig.Delimiter,
Delimiter: csvConfig.Quote,
Terminator: csvConfig.Terminator,
Null: csvConfig.NullString,
BackslashEscape: backslashEscape,
}
csvParser, err := mydump.NewCSVParser(ctx, cfg,
mydump.NewStringReader(string(value)),
int64(lconfig.ReadBlockSize),
worker.NewPool(ctx, 1, "io"), false, nil)
if err != nil {
return nil, err
}
return &batchDecoder{
csvConfig: csvConfig,
data: value,
parser: csvParser,
}, nil
}

// HasNext implements the EventBatchDecoder interface.
func (b *batchDecoder) HasNext() (model.MessageType, bool, error) {
err := b.parser.ReadRow()
if err != nil {
b.closed = true
return model.MessageTypeUnknown, false, err
}

row := b.parser.LastRow()
csvMsg := newCSVMessage(b.csvConfig)
zhaoxinyu marked this conversation as resolved.
Show resolved Hide resolved
if err = csvMsg.decode(row.Row); err != nil {
return model.MessageTypeUnknown, false, errors.Trace(err)
}
b.msg = csvMsg

return model.MessageTypeRow, true, nil
}

// NextResolvedEvent implements the EventBatchDecoder interface.
func (b *batchDecoder) NextResolvedEvent() (uint64, error) {
return 0, nil
}

// NextRowChangedEvent implements the EventBatchDecoder interface.
func (b *batchDecoder) NextRowChangedEvent() (*model.RowChangedEvent, error) {
if b.closed {
return nil, cerror.WrapError(cerror.ErrCSVDecodeFailed, errors.New("no csv row can be found"))
3AceShowHand marked this conversation as resolved.
Show resolved Hide resolved
}
return csvMsg2RowChangedEvent(b.msg), nil
}

// NextDDLEVent implements the EventBatchDecoder interface.
func (b *batchDecoder) NextDDLEvent() (*model.DDLEvent, error) {
return nil, nil
}
53 changes: 53 additions & 0 deletions cdc/sink/codec/csv/csv_decoder_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
// Copyright 2022 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.
package csv

import (
"context"
"testing"

"github.com/pingcap/tiflow/cdc/model"
"github.com/pingcap/tiflow/pkg/config"
"github.com/stretchr/testify/require"
)

func TestCSVBatchDecoder(t *testing.T) {
csvData := `"I","employee","hr",433305438660591626,101,"Smith","Bob","2014-06-04","New York"
"U","employee","hr",433305438660591627,101,"Smith","Bob","2015-10-08","Los Angeles"
"D","employee","hr",433305438660591629,101,"Smith","Bob","2017-03-13","Dallas"
"I","employee","hr",433305438660591630,102,"Alex","Alice","2017-03-14","Shanghai"
"U","employee","hr",433305438660591630,102,"Alex","Alice","2018-06-15","Beijing"
`
ctx := context.Background()
decoder, err := NewBatchDecoder(ctx, &config.CSVConfig{
Delimiter: ",",
Quote: "\"",
Terminator: "\n",
NullString: "\\N",
IncludeCommitTs: true,
}, []byte(csvData))
require.Nil(t, err)

for i := 0; i < 5; i++ {
tp, hasNext, err := decoder.HasNext()
require.Nil(t, err)
require.True(t, hasNext)
require.Equal(t, model.MessageTypeRow, tp)
event, err := decoder.NextRowChangedEvent()
require.Nil(t, err)
require.NotNil(t, event)
}

_, hasNext, _ := decoder.HasNext()
require.False(t, hasNext)
}
4 changes: 2 additions & 2 deletions cdc/sink/codec/csv/csv_encoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,11 @@ func (b *BatchEncoder) AppendRowChangedEvent(
errors.New("no csv config provided"))
}

row, err := buildRowData(b.csvConfig, e)
row, err := rowChangedEvent2CSVMsg(b.csvConfig, e)
if err != nil {
return err
}
b.valueBuf.Write(row)
b.valueBuf.Write(row.encode())
b.batchSize++
if callback != nil {
b.callbackBuf = append(b.callbackBuf, callback)
Expand Down
Loading