github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/sink/dmlsink/mq/dispatcher/partition/columns.go (about)

     1  // Copyright 2023 PingCAP, Inc.
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //     http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // See the License for the specific language governing permissions and
    12  // limitations under the License.
    13  
    14  package partition
    15  
    16  import (
    17  	"strconv"
    18  	"sync"
    19  
    20  	"github.com/pingcap/log"
    21  	"github.com/pingcap/tiflow/cdc/model"
    22  	"github.com/pingcap/tiflow/pkg/errors"
    23  	"github.com/pingcap/tiflow/pkg/hash"
    24  	"go.uber.org/zap"
    25  )
    26  
    27  // ColumnsDispatcher is a partition dispatcher
    28  // which dispatches events based on the given columns.
    29  type ColumnsDispatcher struct {
    30  	hasher *hash.PositionInertia
    31  	lock   sync.Mutex
    32  
    33  	Columns []string
    34  }
    35  
    36  // NewColumnsDispatcher creates a ColumnsDispatcher.
    37  func NewColumnsDispatcher(columns []string) *ColumnsDispatcher {
    38  	return &ColumnsDispatcher{
    39  		hasher:  hash.NewPositionInertia(),
    40  		Columns: columns,
    41  	}
    42  }
    43  
    44  // DispatchRowChangedEvent returns the target partition to which
    45  // a row changed event should be dispatched.
    46  func (r *ColumnsDispatcher) DispatchRowChangedEvent(row *model.RowChangedEvent, partitionNum int32) (int32, string, error) {
    47  	r.lock.Lock()
    48  	defer r.lock.Unlock()
    49  	r.hasher.Reset()
    50  
    51  	r.hasher.Write([]byte(row.TableInfo.GetSchemaName()), []byte(row.TableInfo.GetTableName()))
    52  
    53  	dispatchCols := row.Columns
    54  	if len(dispatchCols) == 0 {
    55  		dispatchCols = row.PreColumns
    56  	}
    57  
    58  	offsets, ok := row.TableInfo.OffsetsByNames(r.Columns)
    59  	if !ok {
    60  		log.Error("columns not found when dispatch event",
    61  			zap.Any("tableName", row.TableInfo.GetTableName()),
    62  			zap.Strings("columns", r.Columns))
    63  		return 0, "", errors.ErrDispatcherFailed.GenWithStack(
    64  			"columns not found when dispatch event, table: %v, columns: %v", row.TableInfo.GetTableName(), r.Columns)
    65  	}
    66  
    67  	for idx := 0; idx < len(r.Columns); idx++ {
    68  		col := dispatchCols[offsets[idx]]
    69  		if col == nil {
    70  			continue
    71  		}
    72  		r.hasher.Write([]byte(r.Columns[idx]), []byte(model.ColumnValueString(col.Value)))
    73  	}
    74  
    75  	sum32 := r.hasher.Sum32()
    76  	return int32(sum32 % uint32(partitionNum)), strconv.FormatInt(int64(sum32), 10), nil
    77  }