forked from pingcap/tiflow
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
kafka(ticdc): support columns dispatcher. (pingcap#9863)
close pingcap#9862
- Loading branch information
1 parent
3b8d55b
commit 3802006
Showing
15 changed files
with
436 additions
and
22 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
// Copyright 2023 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 partition | ||
|
||
import ( | ||
"strconv" | ||
"sync" | ||
|
||
"github.com/pingcap/log" | ||
"github.com/pingcap/tiflow/cdc/model" | ||
"github.com/pingcap/tiflow/pkg/errors" | ||
"github.com/pingcap/tiflow/pkg/hash" | ||
"go.uber.org/zap" | ||
) | ||
|
||
// ColumnsDispatcher is a partition dispatcher | ||
// which dispatches events based on the given columns. | ||
type ColumnsDispatcher struct { | ||
hasher *hash.PositionInertia | ||
lock sync.Mutex | ||
|
||
Columns []string | ||
} | ||
|
||
// NewColumnsDispatcher creates a ColumnsDispatcher. | ||
func NewColumnsDispatcher(columns []string) *ColumnsDispatcher { | ||
return &ColumnsDispatcher{ | ||
hasher: hash.NewPositionInertia(), | ||
Columns: columns, | ||
} | ||
} | ||
|
||
// DispatchRowChangedEvent returns the target partition to which | ||
// a row changed event should be dispatched. | ||
func (r *ColumnsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string, error) { | ||
r.lock.Lock() | ||
defer r.lock.Unlock() | ||
r.hasher.Reset() | ||
|
||
r.hasher.Write([]byte(row.Table.Schema), []byte(row.Table.Table)) | ||
|
||
dispatchCols := row.Columns | ||
if len(dispatchCols) == 0 { | ||
dispatchCols = row.PreColumns | ||
} | ||
|
||
offsets, ok := row.TableInfo.ColumnsByNames(r.Columns) | ||
if !ok { | ||
log.Error("columns not found when dispatch event", | ||
zap.Any("tableName", row.Table), | ||
zap.Strings("columns", r.Columns)) | ||
return 0, "", errors.ErrDispatcherFailed.GenWithStack( | ||
"columns not found when dispatch event, table: %v, columns: %v", row.Table, r.Columns) | ||
} | ||
|
||
for idx := 0; idx < len(r.Columns); idx++ { | ||
r.hasher.Write([]byte(r.Columns[idx]), []byte(model.ColumnValueString(dispatchCols[offsets[idx]].Value))) | ||
} | ||
|
||
sum32 := r.hasher.Sum32() | ||
return int32(sum32 % uint32(partitionNum)), strconv.FormatInt(int64(sum32), 10), nil | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,81 @@ | ||
// Copyright 2023 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 partition | ||
|
||
import ( | ||
"testing" | ||
|
||
timodel "github.com/pingcap/tidb/parser/model" | ||
"github.com/pingcap/tiflow/cdc/model" | ||
"github.com/pingcap/tiflow/pkg/errors" | ||
"github.com/stretchr/testify/require" | ||
) | ||
|
||
func TestColumnsDispatcher(t *testing.T) { | ||
t.Parallel() | ||
|
||
event := &model.RowChangedEvent{ | ||
Table: &model.TableName{ | ||
Schema: "test", | ||
Table: "t1", | ||
}, | ||
TableInfo: &model.TableInfo{ | ||
TableInfo: &timodel.TableInfo{ | ||
Columns: []*timodel.ColumnInfo{ | ||
{ | ||
Name: timodel.CIStr{ | ||
O: "col2", | ||
}, | ||
Offset: 1, | ||
}, | ||
{ | ||
Name: timodel.CIStr{ | ||
O: "col1", | ||
}, | ||
Offset: 0, | ||
}, | ||
{ | ||
Name: timodel.CIStr{ | ||
O: "col3", | ||
}, | ||
Offset: 2, | ||
}, | ||
}, | ||
}, | ||
}, | ||
Columns: []*model.Column{ | ||
{ | ||
Name: "col1", | ||
Value: 11, | ||
}, | ||
{ | ||
Name: "col2", | ||
Value: 22, | ||
}, | ||
{ | ||
Name: "col3", | ||
Value: 33, | ||
}, | ||
}, | ||
} | ||
|
||
p := NewColumnsDispatcher([]string{"col-2", "col-not-found"}) | ||
_, _, err := p.DispatchRowChangedEvent(event, 16) | ||
require.ErrorIs(t, err, errors.ErrDispatcherFailed) | ||
|
||
p = NewColumnsDispatcher([]string{"col2", "col1"}) | ||
index, _, err := p.DispatchRowChangedEvent(event, 16) | ||
require.NoError(t, err) | ||
require.Equal(t, int32(15), index) | ||
} |
Oops, something went wrong.