github.com/matrixorigin/matrixone@v1.2.0/pkg/stream/connector/connector.go (about)

     1  // Copyright 2023 Matrix Origin
     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  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package moconnector
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"strconv"
    21  	"strings"
    22  	"sync"
    23  	"time"
    24  
    25  	"github.com/confluentinc/confluent-kafka-go/v2/kafka"
    26  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    27  	"github.com/matrixorigin/matrixone/pkg/defines"
    28  	"github.com/matrixorigin/matrixone/pkg/pb/task"
    29  	mokafka "github.com/matrixorigin/matrixone/pkg/stream/adapter/kafka"
    30  	"github.com/matrixorigin/matrixone/pkg/taskservice"
    31  	ie "github.com/matrixorigin/matrixone/pkg/util/internalExecutor"
    32  	"go.uber.org/zap"
    33  )
    34  
    35  func getBufferLimit(bufferLimitStr string) int {
    36  	bufferLimit, err := strconv.Atoi(bufferLimitStr) // Convert the string to an integer
    37  	if err != nil {
    38  		// Handle the error, perhaps set to default if the conversion fails
    39  		bufferLimit = 1
    40  	}
    41  	return bufferLimit
    42  }
    43  
    44  func getTimeWindow(timeWindowStr string) int {
    45  	timeWindow, err := strconv.Atoi(timeWindowStr) // Convert the string to an integer
    46  	if err != nil {
    47  		// Handle the error, perhaps set to default if the conversion fails
    48  		timeWindow = 1000 // 1second, 1000ms
    49  	}
    50  	return timeWindow
    51  }
    52  
    53  func KafkaSinkConnectorExecutor(
    54  	logger *zap.Logger,
    55  	ts taskservice.TaskService,
    56  	ieFactory func() ie.InternalExecutor,
    57  	attachToTask func(context.Context, uint64, taskservice.ActiveRoutine) error,
    58  ) func(context.Context, task.Task) error {
    59  	return func(ctx context.Context, t task.Task) error {
    60  		ctx1, cancel := context.WithTimeout(context.Background(), time.Second*3)
    61  		defer cancel()
    62  		tasks, err := ts.QueryDaemonTask(ctx1,
    63  			taskservice.WithTaskIDCond(taskservice.EQ, t.GetID()),
    64  		)
    65  		if err != nil {
    66  			return err
    67  		}
    68  		if len(tasks) != 1 {
    69  			return moerr.NewInternalError(ctx, "invalid tasks count %d", len(tasks))
    70  		}
    71  		details, ok := tasks[0].Details.Details.(*task.Details_Connector)
    72  		if !ok {
    73  			return moerr.NewInternalError(ctx, "invalid details type")
    74  		}
    75  		options := details.Connector.Options
    76  		fullTableName := details.Connector.TableName
    77  		// Set database and table name for options.
    78  		ss := strings.Split(fullTableName, ".")
    79  		options[mokafka.DatabaseKey] = ss[0]
    80  		options[mokafka.TableKey] = ss[1]
    81  		options[mokafka.CREATED_AT] = tasks[0].CreateAt.String()
    82  		bufferLimit := getBufferLimit(options[mokafka.BufferLimitKey])
    83  
    84  		c, err := NewKafkaMoConnector(logger, options, ieFactory(), bufferLimit)
    85  		if err != nil {
    86  			return err
    87  		}
    88  		if err := attachToTask(ctx, t.GetID(), c); err != nil {
    89  			return err
    90  		}
    91  		// Start the connector task and hangs here.
    92  		if err := c.Start(ctx); err != nil {
    93  			return err
    94  		}
    95  		return nil
    96  	}
    97  }
    98  
    99  // KafkaMoConnector is an example implementation of the Connector interface for a Kafka to MO Table connection.
   100  
   101  type KafkaMoConnector struct {
   102  	logger       *zap.Logger
   103  	kafkaAdapter mokafka.KafkaAdapterInterface
   104  	options      map[string]string
   105  	ie           ie.InternalExecutor
   106  	decoder      Decoder
   107  	converter    Converter
   108  	resumeC      chan struct{}
   109  	cancelC      chan struct{}
   110  	pauseC       chan struct{}
   111  	bufferLimit  int
   112  }
   113  
   114  func convertToKafkaConfig(configs map[string]string) *kafka.ConfigMap {
   115  	kafkaConfigs := &kafka.ConfigMap{}
   116  	allowedKeys := map[string]struct{}{
   117  		"bootstrap.servers": {},
   118  		"security.protocol": {},
   119  		"sasl.mechanisms":   {},
   120  		"sasl.username":     {},
   121  		"sasl.password":     {},
   122  		// Add other Kafka-specific properties here...
   123  	}
   124  
   125  	for key, value := range configs {
   126  		if _, ok := allowedKeys[key]; ok {
   127  			kafkaConfigs.SetKey(key, value)
   128  		}
   129  	}
   130  	groupId := configs[mokafka.TopicKey] + "-" + configs[mokafka.DatabaseKey] + "-" + configs[mokafka.TableKey] + "-" + configs[mokafka.PartitionKey] + "-" + configs[mokafka.CREATED_AT]
   131  	kafkaConfigs.SetKey("group.id", groupId)
   132  	return kafkaConfigs
   133  }
   134  
   135  func NewKafkaMoConnector(logger *zap.Logger, options map[string]string, ie ie.InternalExecutor, buffer_limit int) (*KafkaMoConnector, error) {
   136  	// Validate options before proceeding
   137  	kmc := &KafkaMoConnector{
   138  		logger:      logger,
   139  		options:     options,
   140  		ie:          ie,
   141  		decoder:     newJsonDecoder(),
   142  		bufferLimit: buffer_limit,
   143  	}
   144  	if err := kmc.validateParams(); err != nil {
   145  		return nil, err
   146  	}
   147  	kmc.converter = newSQLConverter(options[mokafka.DatabaseKey], options[mokafka.TableKey])
   148  
   149  	// Create a Kafka consumer using the provided options
   150  	kafkaAdapter, err := mokafka.NewKafkaAdapter(convertToKafkaConfig(options))
   151  	if err != nil {
   152  		return nil, err
   153  	}
   154  
   155  	kmc.kafkaAdapter = kafkaAdapter
   156  	kmc.resumeC = make(chan struct{})
   157  	kmc.cancelC = make(chan struct{})
   158  	kmc.pauseC = make(chan struct{})
   159  	return kmc, nil
   160  }
   161  
   162  func (k *KafkaMoConnector) validateParams() error {
   163  	// 1. Check mandatory fields
   164  	mandatoryFields := []string{
   165  		"type", "topic", "value",
   166  		"bootstrap.servers",
   167  	}
   168  
   169  	for _, field := range mandatoryFields {
   170  		if _, exists := k.options[field]; !exists || k.options[field] == "" {
   171  			return moerr.NewInternalError(context.Background(), "missing required params")
   172  		}
   173  	}
   174  
   175  	// 2. Check for valid type
   176  	if k.options["type"] != "kafka" {
   177  		return moerr.NewInternalError(context.Background(), "Invalid connector type")
   178  	}
   179  
   180  	// 3. Check for supported value format
   181  	if k.options["value"] != "json" {
   182  		return moerr.NewInternalError(context.Background(), "Unsupported value format")
   183  	}
   184  
   185  	return nil
   186  }
   187  
   188  // Start begins consuming messages from Kafka and writing them to the MO Table.
   189  func (k *KafkaMoConnector) Start(ctx context.Context) error {
   190  	if k.kafkaAdapter == nil {
   191  		return moerr.NewInternalError(ctx, "Kafka Adapter not initialized")
   192  	}
   193  
   194  	ct, err := k.kafkaAdapter.GetKafkaConsumer()
   195  
   196  	if err != nil {
   197  		return moerr.NewInternalError(ctx, "Kafka Adapter Consumer not initialized")
   198  	}
   199  	// Define the topic to consume from
   200  	topic := k.options[mokafka.TopicKey]
   201  
   202  	// Subscribe to the topic
   203  	if k.options[mokafka.PartitionKey] != "" {
   204  		partition, err := strconv.Atoi(k.options[mokafka.PartitionKey])
   205  		if err != nil {
   206  			return moerr.NewInternalError(ctx, "Invalid partition")
   207  		}
   208  		if err := ct.Assign([]kafka.TopicPartition{{Topic: &topic, Partition: int32(partition)}}); err != nil {
   209  			return moerr.NewInternalError(ctx, "Failed to assign partition")
   210  		}
   211  	} else {
   212  		if err := ct.Subscribe(topic, nil); err != nil {
   213  			return moerr.NewInternalError(ctx, "Failed to subscribe to topic")
   214  		}
   215  	}
   216  	// Continuously listen for messages
   217  	var buffered_messages []*kafka.Message
   218  	timeWindow := getTimeWindow(k.options[mokafka.TimeWindowKey])
   219  	var timer *time.Timer
   220  	timer = time.NewTimer(time.Duration(timeWindow) * time.Millisecond)
   221  	timerRunning := false
   222  	var mutex sync.Mutex
   223  
   224  	for {
   225  		select {
   226  		case <-ctx.Done():
   227  			return nil
   228  
   229  		case <-k.cancelC:
   230  			timer.Stop()
   231  			return ct.Close()
   232  
   233  		case <-k.pauseC:
   234  			select {
   235  			case <-ctx.Done():
   236  				return nil
   237  			case <-k.cancelC:
   238  				return nil
   239  			case <-k.resumeC:
   240  			}
   241  
   242  		default:
   243  			if ct.IsClosed() {
   244  				return nil
   245  			}
   246  			ev := ct.Poll(100)
   247  			if ev == nil {
   248  				continue
   249  			}
   250  
   251  			switch e := ev.(type) {
   252  			case *kafka.Message:
   253  				if e.Value == nil {
   254  					continue
   255  				}
   256  				mutex.Lock()
   257  				buffered_messages = append(buffered_messages, e)
   258  
   259  				// Start the timer if it's not already running
   260  				if !timerRunning {
   261  					timer.Stop()
   262  					timer = time.AfterFunc(time.Duration(timeWindow)*time.Millisecond, func() {
   263  						mutex.Lock()
   264  						if len(buffered_messages) > 0 {
   265  							k.insertRow(buffered_messages)
   266  							buffered_messages = buffered_messages[:0]
   267  						}
   268  						timerRunning = false
   269  						mutex.Unlock()
   270  					})
   271  					timerRunning = true
   272  				}
   273  
   274  				// Flush the buffer if the limit is reached
   275  				if len(buffered_messages) >= k.bufferLimit {
   276  					if timerRunning {
   277  						timer.Stop()
   278  						timerRunning = false
   279  					}
   280  					k.insertRow(buffered_messages)
   281  					buffered_messages = buffered_messages[:0]
   282  				}
   283  				mutex.Unlock()
   284  			case kafka.Error:
   285  				// Handle the error accordingly.
   286  				k.logger.Error("got error message", zap.Error(e))
   287  			default:
   288  				// Ignored other types of events
   289  			}
   290  		}
   291  	}
   292  }
   293  
   294  // Resume implements the taskservice.ActiveRoutine interface.
   295  func (k *KafkaMoConnector) Resume() error {
   296  	k.resumeC <- struct{}{}
   297  	return nil
   298  }
   299  
   300  // Pause implements the taskservice.ActiveRoutine interface.
   301  func (k *KafkaMoConnector) Pause() error {
   302  	k.pauseC <- struct{}{}
   303  	return nil
   304  }
   305  
   306  // Cancel implements the taskservice.ActiveRoutine interface.
   307  func (k *KafkaMoConnector) Cancel() error {
   308  	// Cancel the connector go-routine.
   309  	close(k.cancelC)
   310  	return nil
   311  }
   312  
   313  func (k *KafkaMoConnector) Close() error {
   314  	// Close the Kafka consumer.
   315  	ct, err := k.kafkaAdapter.GetKafkaConsumer()
   316  	if err != nil {
   317  		return moerr.NewInternalError(context.Background(), "Kafka Adapter Consumer not initialized")
   318  	}
   319  	if err := ct.Close(); err != nil {
   320  		return moerr.NewInternalError(context.Background(), "Error closing Kafka consumer")
   321  	}
   322  	return nil
   323  }
   324  
   325  func (k *KafkaMoConnector) insertRow(msgs []*kafka.Message) {
   326  	opts := ie.SessionOverrideOptions{}
   327  	ctx := context.Background()
   328  	sql := k.options["sql"]
   329  	dbName := k.options[mokafka.DatabaseKey]
   330  	tableName := k.options[mokafka.TableKey]
   331  	if sql == "" {
   332  		return
   333  	}
   334  	ctx = context.WithValue(ctx, defines.SourceScanResKey{}, msgs)
   335  
   336  	sql = fmt.Sprintf("USE %s; INSERT INTO %s.%s %s ",
   337  		dbName, dbName, tableName, sql)
   338  	err := k.ie.Exec(ctx, sql, opts)
   339  	if err != nil {
   340  		k.logger.Error("failed to insert row", zap.String("SQL", sql), zap.Error(err))
   341  	}
   342  }