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

     1  // Copyright 2021 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 mokafka
    16  
    17  import (
    18  	"bytes"
    19  	"context"
    20  	"encoding/binary"
    21  	"encoding/json"
    22  	"errors"
    23  	"fmt"
    24  	"math"
    25  	"strconv"
    26  	"sync"
    27  	"time"
    28  
    29  	"github.com/confluentinc/confluent-kafka-go/v2/kafka"
    30  	"github.com/confluentinc/confluent-kafka-go/v2/schemaregistry"
    31  	"github.com/gogo/protobuf/proto"
    32  	"github.com/google/uuid"
    33  	"github.com/jhump/protoreflect/desc"
    34  	"github.com/jhump/protoreflect/desc/protoparse"
    35  	"github.com/jhump/protoreflect/dynamic"
    36  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    37  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    38  	"github.com/matrixorigin/matrixone/pkg/container/batch"
    39  	"github.com/matrixorigin/matrixone/pkg/container/nulls"
    40  	"github.com/matrixorigin/matrixone/pkg/container/types"
    41  	"github.com/matrixorigin/matrixone/pkg/container/vector"
    42  	"github.com/matrixorigin/matrixone/pkg/logutil"
    43  )
    44  
    45  type ValueType string
    46  
    47  const (
    48  	TypeKey  = "type"
    49  	TopicKey = "topic"
    50  
    51  	DatabaseKey = "database"
    52  
    53  	TimeWindowKey = "time_window"
    54  
    55  	BufferLimitKey = "buffer_limit"
    56  
    57  	TableKey = "table"
    58  
    59  	ValueKey = "value"
    60  
    61  	PartitionKey        = "partition"
    62  	RelkindKey          = "relkind"
    63  	BootstrapServersKey = "bootstrap.servers"
    64  	ProtobufSchemaKey   = "protobuf.schema"
    65  	ProtobufMessagekey  = "protobuf.message"
    66  
    67  	SchemaRegistryKey = "schema.registry"
    68  
    69  	JSON       ValueType = "json"
    70  	AVRO       ValueType = "avro"
    71  	PROTOBUF   ValueType = "protobuf"
    72  	PROTOBUFSR ValueType = "protobuf_sr"
    73  
    74  	CREATED_AT = "created_at"
    75  )
    76  
    77  type DataGetter interface {
    78  	GetFieldValue(name string) (interface{}, bool)
    79  }
    80  
    81  type JsonDataGetter struct {
    82  	Key   []byte
    83  	Value []byte
    84  	Data  map[string]interface{} // Cache the parsed JSON for efficiency
    85  }
    86  
    87  func (j *JsonDataGetter) GetFieldValue(name string) (interface{}, bool) {
    88  	// If the JSON data hasn't been parsed, do it now
    89  	if j.Data == nil {
    90  		err := json.Unmarshal(j.Value, &j.Data)
    91  		if err != nil {
    92  			return nil, false
    93  		}
    94  	}
    95  
    96  	val, ok := j.Data[name]
    97  	return val, ok
    98  }
    99  
   100  type ProtoDataGetter struct {
   101  	Value *dynamic.Message
   102  	Key   any
   103  }
   104  
   105  func (p *ProtoDataGetter) GetFieldValue(name string) (interface{}, bool) {
   106  	val := p.Value.GetFieldByName(name)
   107  	return val, val != nil
   108  }
   109  
   110  type KafkaAdapterInterface interface {
   111  	InitSchemaRegistry(url string) error
   112  	Close()
   113  	CreateTopic(ctx context.Context, topicName string, partitions int, replicationFactor int) error
   114  	DescribeTopicDetails(ctx context.Context, topicName string) (*kafka.TopicMetadata, error)
   115  	ReadMessagesFromPartition(topic string, partition int32, offset int64, limit int) ([]*kafka.Message, error)
   116  	ReadMessagesFromTopic(topic string, offset int64, limit int64, configs map[string]interface{}) ([]*kafka.Message, error)
   117  	GetSchemaForTopic(topic string, isKey bool) (schemaregistry.SchemaMetadata, error)
   118  
   119  	GetKafkaConsumer() (*kafka.Consumer, error)
   120  	ProduceMessage(topic string, key, value []byte) (int64, error)
   121  }
   122  type KafkaAdapter struct {
   123  	Producer       *kafka.Producer
   124  	Consumer       *kafka.Consumer
   125  	AdminClient    *kafka.AdminClient
   126  	SchemaRegistry schemaregistry.Client
   127  	ConfigMap      *kafka.ConfigMap
   128  	Connected      bool
   129  }
   130  
   131  func (ka *KafkaAdapter) InitSchemaRegistry(url string) error {
   132  	client, err := schemaregistry.NewClient(schemaregistry.NewConfig(url))
   133  	if err != nil {
   134  		return err
   135  	}
   136  	ka.SchemaRegistry = client
   137  	return nil
   138  }
   139  
   140  func NewKafkaAdapter(configMap *kafka.ConfigMap) (KafkaAdapterInterface, error) {
   141  	// Create a new admin client instance
   142  	adminClient, err := kafka.NewAdminClient(configMap)
   143  	if err != nil {
   144  		return nil, err
   145  	}
   146  
   147  	// Create a new consumer client instance
   148  	//todo : better handle the offset reset
   149  	configMap.SetKey("auto.offset.reset", "earliest")
   150  	consumer, err := kafka.NewConsumer(configMap)
   151  	if err != nil {
   152  		return nil, moerr.NewInternalError(context.Background(), fmt.Sprintf("unable to create confluent consumer client: %s", err))
   153  	}
   154  
   155  	// Create a new producer client instance
   156  	producer, err := kafka.NewProducer(configMap)
   157  	if err != nil {
   158  		return nil, moerr.NewInternalError(context.Background(), fmt.Sprintf("unable to create confluent producer client: %s", err))
   159  	}
   160  
   161  	// Return a new KafkaAdapter instance
   162  	return &KafkaAdapter{
   163  		Producer:    producer,
   164  		AdminClient: adminClient,
   165  		Consumer:    consumer,
   166  		ConfigMap:   configMap,
   167  		Connected:   true,
   168  	}, nil
   169  }
   170  
   171  func (ka *KafkaAdapter) GetKafkaConsumer() (*kafka.Consumer, error) {
   172  	return ka.Consumer, nil
   173  }
   174  func (ka *KafkaAdapter) Close() {
   175  
   176  	// Close the Producer if it's initialized
   177  	if ka.Producer != nil {
   178  		ka.Producer.Close()
   179  	}
   180  
   181  	// Close the Consumer if it's initialized
   182  	if ka.Consumer != nil {
   183  		ka.Consumer.Close()
   184  	}
   185  
   186  	// Close the AdminClient if it's initialized
   187  	if ka.AdminClient != nil {
   188  		ka.AdminClient.Close()
   189  	}
   190  
   191  	// Update the Connected status
   192  	ka.Connected = false
   193  }
   194  
   195  func (ka *KafkaAdapter) CreateTopic(ctx context.Context, topicName string, partitions int, replicationFactor int) error {
   196  	topicSpecification := kafka.TopicSpecification{
   197  		Topic:             topicName,
   198  		NumPartitions:     partitions,
   199  		ReplicationFactor: replicationFactor,
   200  		// can add more configs here
   201  	}
   202  
   203  	results, err := ka.AdminClient.CreateTopics(ctx, []kafka.TopicSpecification{topicSpecification})
   204  	if err != nil {
   205  		return err
   206  	}
   207  
   208  	// Check results for errors
   209  	for _, result := range results {
   210  		if result.Error.Code() != kafka.ErrNoError {
   211  			return result.Error
   212  		}
   213  	}
   214  	return nil
   215  }
   216  
   217  func (ka *KafkaAdapter) DescribeTopicDetails(ctx context.Context, topicName string) (*kafka.TopicMetadata, error) {
   218  
   219  	// Fetch metadata
   220  	meta, err := ka.AdminClient.GetMetadata(&topicName, false, int(10*time.Second.Milliseconds()))
   221  	if err != nil {
   222  		return nil, err
   223  	}
   224  
   225  	// Find and return the topic's metadata
   226  	for _, topic := range meta.Topics {
   227  		if topic.Topic == topicName {
   228  			return &topic, nil
   229  		}
   230  	}
   231  
   232  	return nil, moerr.NewInternalError(ctx, "topic not found")
   233  }
   234  
   235  func (ka *KafkaAdapter) ReadMessagesFromPartition(topic string, partition int32, offset int64, limit int) ([]*kafka.Message, error) {
   236  	if ka.Consumer == nil {
   237  		return nil, moerr.NewInternalError(context.Background(), "consumer not initialized")
   238  	}
   239  
   240  	// Assign the specific partition with the desired offset
   241  	err := ka.Consumer.Assign([]kafka.TopicPartition{
   242  		{Topic: &topic, Partition: partition, Offset: kafka.Offset(offset)},
   243  	})
   244  	if err != nil {
   245  		return nil, moerr.NewInternalError(context.Background(), fmt.Sprintf("failed to assign partition: %s", err))
   246  	}
   247  
   248  	var messages []*kafka.Message
   249  	for i := 0; i < limit; i++ {
   250  		msg, err := ka.Consumer.ReadMessage(-1) // Wait indefinitely until a message is available
   251  		if err != nil {
   252  			// Check for timeout
   253  			if kafkaErr, ok := err.(kafka.Error); ok && kafkaErr.Code() == kafka.ErrTimedOut {
   254  				break // Exit the loop if a timeout occurs
   255  			} else {
   256  				return nil, moerr.NewInternalError(context.Background(), fmt.Sprintf("failed to read message: %s", err))
   257  			}
   258  		}
   259  		messages = append(messages, msg)
   260  	}
   261  
   262  	return messages, nil
   263  }
   264  
   265  func (ka *KafkaAdapter) ReadMessagesFromTopic(topic string, offset int64, limit int64, configs map[string]interface{}) ([]*kafka.Message, error) {
   266  	if ka.Consumer == nil {
   267  		return nil, moerr.NewInternalError(context.Background(), "consumer not initialized")
   268  	}
   269  
   270  	// Fetch metadata to get all partitions
   271  	meta, err := ka.Consumer.GetMetadata(&topic, false, -1) // timeout in ms
   272  	if err != nil {
   273  		return nil, err
   274  	}
   275  
   276  	topicMetadata, ok := meta.Topics[topic]
   277  	if !ok {
   278  		return nil, moerr.NewInternalError(context.Background(), "topic not found in metadata")
   279  	}
   280  
   281  	var messages []*kafka.Message
   282  	var partitions []kafka.PartitionMetadata
   283  	if configs[PartitionKey] != nil {
   284  		partition, err := strconv.Atoi(configs[PartitionKey].(string))
   285  		if err != nil {
   286  			return nil, err
   287  		}
   288  		for _, p := range topicMetadata.Partitions {
   289  			if p.ID == int32(partition) {
   290  				partitions = append(partitions, p)
   291  				break
   292  			}
   293  		}
   294  	} else {
   295  		partitions = topicMetadata.Partitions
   296  	}
   297  	for _, p := range partitions {
   298  		// Fetch the high watermark for the partition
   299  		_, highwatermarkHigh, err := ka.Consumer.QueryWatermarkOffsets(topic, p.ID, -1)
   300  		if err != nil {
   301  			return nil, err
   302  		}
   303  
   304  		// Calculate the number of messages available to consume
   305  		availableMessages := highwatermarkHigh - offset
   306  		if availableMessages <= 0 {
   307  			continue
   308  		}
   309  
   310  		// Determine the number of messages to consume from this partition
   311  		partitionLimit := limit - int64(len(messages))
   312  		if partitionLimit > availableMessages {
   313  			partitionLimit = availableMessages
   314  		}
   315  		if limit == 0 {
   316  			partitionLimit = availableMessages
   317  		}
   318  
   319  		// Assign the specific partition with the desired offset
   320  		err = ka.Consumer.Assign([]kafka.TopicPartition{
   321  			{Topic: &topic, Partition: p.ID, Offset: kafka.Offset(offset)},
   322  		})
   323  		if err != nil {
   324  			return nil, err
   325  		}
   326  
   327  		for i := int64(0); i < partitionLimit; i++ {
   328  			msg, err := ka.Consumer.ReadMessage(-1)
   329  			if err != nil {
   330  				// Check for timeout
   331  				var kafkaErr kafka.Error
   332  				if errors.As(err, &kafkaErr) && kafkaErr.Code() == kafka.ErrTimedOut {
   333  					break // Exit the loop if a timeout occurs
   334  				}
   335  			}
   336  			messages = append(messages, msg)
   337  		}
   338  	}
   339  
   340  	return messages, nil
   341  }
   342  
   343  func (ka *KafkaAdapter) BatchRead(topic string, startOffset int64, limit int, batchSize int) ([]*kafka.Message, error) {
   344  	// Fetch metadata to get all partitions
   345  	meta, err := ka.Consumer.GetMetadata(&topic, false, 5000) // timeout in ms
   346  	if err != nil {
   347  		return nil, err
   348  	}
   349  
   350  	topicMetadata, ok := meta.Topics[topic]
   351  	if !ok {
   352  		return nil, moerr.NewInternalError(context.Background(), "topic not found in metadata")
   353  	}
   354  
   355  	numGoroutines := (limit + batchSize - 1) / batchSize
   356  
   357  	messagesCh := make(chan []*kafka.Message, numGoroutines)
   358  	errCh := make(chan error, numGoroutines)
   359  	var wg sync.WaitGroup
   360  
   361  	// Loop over each partition and start goroutines for reading
   362  	for _, p := range topicMetadata.Partitions {
   363  		wg.Add(1)
   364  		go func(partition int32) {
   365  			defer wg.Done()
   366  
   367  			// Read a batch of messages
   368  			messages, err := ka.ReadMessagesFromPartition(topic, partition, startOffset, batchSize)
   369  			if err != nil {
   370  				errCh <- err
   371  				return
   372  			}
   373  			messagesCh <- messages
   374  		}(p.ID)
   375  	}
   376  
   377  	// Wait for all goroutines to finish
   378  	wg.Wait()
   379  
   380  	close(messagesCh)
   381  	close(errCh)
   382  
   383  	// Collect all messages
   384  	var allMessages []*kafka.Message
   385  	for batch := range messagesCh {
   386  		allMessages = append(allMessages, batch...)
   387  	}
   388  
   389  	// Return the first error encountered, if any
   390  	for err := range errCh {
   391  		return nil, err
   392  	}
   393  
   394  	return allMessages, nil
   395  }
   396  
   397  func (ka *KafkaAdapter) GetSchemaForTopic(topic string, isKey bool) (schemaregistry.SchemaMetadata, error) {
   398  	if ka.SchemaRegistry == nil {
   399  		return schemaregistry.SchemaMetadata{}, moerr.NewInternalError(context.Background(), "schema registry not initialized")
   400  	}
   401  
   402  	subjectSuffix := "value"
   403  	if isKey {
   404  		subjectSuffix = "key"
   405  	}
   406  	subject := fmt.Sprintf("%s-%s", topic, subjectSuffix)
   407  
   408  	// Fetch the schema for the subject
   409  	return ka.SchemaRegistry.GetLatestSchemaMetadata(subject)
   410  }
   411  
   412  func (ka *KafkaAdapter) ProduceMessage(topic string, key, value []byte) (int64, error) {
   413  
   414  	deliveryChan := make(chan kafka.Event)
   415  	defer close(deliveryChan)
   416  
   417  	message := &kafka.Message{
   418  		TopicPartition: kafka.TopicPartition{Topic: &topic},
   419  		Key:            key,
   420  		Value:          value,
   421  	}
   422  
   423  	err := ka.Producer.Produce(message, deliveryChan)
   424  	if err != nil {
   425  		return -1, moerr.NewInternalError(context.Background(), fmt.Sprintf("failed to produce message: %s", err))
   426  	}
   427  
   428  	e := <-deliveryChan
   429  	m := e.(*kafka.Message)
   430  	if m.TopicPartition.Error != nil {
   431  		return -1, m.TopicPartition.Error
   432  	}
   433  
   434  	return int64(m.TopicPartition.Offset), nil
   435  }
   436  
   437  func newBatch(batchSize int, typs []types.Type, pool *mpool.MPool) (*batch.Batch, error) {
   438  
   439  	//alloc space for vector
   440  	batch := batch.NewWithSize(len(typs))
   441  	for i, typ := range typs {
   442  		typ.Size = int32(typ.Oid.TypeLen())
   443  		vec := vector.NewVec(typ)
   444  		err := vec.PreExtend(batchSize, pool)
   445  		if err != nil {
   446  			return nil, err
   447  		}
   448  		vec.SetLength(batchSize)
   449  		batch.Vecs[i] = vec
   450  	}
   451  	return batch, nil
   452  }
   453  
   454  func PopulateBatchFromMSG(ctx context.Context, ka KafkaAdapterInterface, typs []types.Type, attrKeys []string, msgs []*kafka.Message, configs map[string]interface{}, mp *mpool.MPool) (*batch.Batch, error) {
   455  	b, err := newBatch(len(msgs), typs, mp)
   456  	if err != nil {
   457  		return nil, err
   458  	}
   459  	unexpectEOF := false
   460  	value, ok := configs[ValueKey].(string)
   461  	if !ok {
   462  		return nil, moerr.NewInternalError(ctx, "expected string value for key: %s", ValueKey)
   463  	}
   464  	switch ValueType(value) {
   465  	case JSON:
   466  		for i, msg := range msgs {
   467  			err := populateOneRowData(ctx, b, attrKeys, &JsonDataGetter{Key: msg.Key, Value: msg.Value}, i, typs, mp)
   468  			if err != nil {
   469  				logutil.Error("populate row failed")
   470  			}
   471  		}
   472  	case PROTOBUF:
   473  		md, err := convertProtobufSchemaToMD(configs["protobuf.schema"].(string), configs["protobuf.message"].(string))
   474  		if err != nil {
   475  			return nil, err
   476  		}
   477  		for i, msg := range msgs {
   478  			msgValue, err := deserializeProtobuf(md, msg.Value, false)
   479  			if err != nil {
   480  				return nil, err
   481  			}
   482  			err = populateOneRowData(ctx, b, attrKeys, &ProtoDataGetter{Value: msgValue, Key: msg.Key}, i, typs, mp)
   483  			if err != nil {
   484  				return nil, err
   485  			}
   486  		}
   487  
   488  	case PROTOBUFSR:
   489  		schema, err := ka.GetSchemaForTopic(configs[TopicKey].(string), false)
   490  		if err != nil {
   491  			return nil, err
   492  		}
   493  		md, err := convertProtobufSchemaToMD(schema.Schema, schema.SchemaInfo.Schema)
   494  		if err != nil {
   495  			return nil, err
   496  		}
   497  		for i, msg := range msgs {
   498  			msgValue, _ := deserializeProtobuf(md, msg.Value, true)
   499  			err := populateOneRowData(ctx, b, attrKeys, &ProtoDataGetter{Value: msgValue, Key: msg.Key}, i, typs, mp)
   500  			if err != nil {
   501  				return nil, err
   502  			}
   503  		}
   504  	default:
   505  		return nil, moerr.NewInternalError(ctx, "Unsupported value for key: %s", ValueKey)
   506  	}
   507  
   508  	n := b.Vecs[0].Length()
   509  	if unexpectEOF && n > 0 {
   510  		n--
   511  		for i := 0; i < b.VectorCount(); i++ {
   512  			vec := b.GetVector(int32(i))
   513  			vec.SetLength(n)
   514  		}
   515  	}
   516  	b.SetRowCount(n)
   517  	return b, nil
   518  }
   519  func populateOneRowData(ctx context.Context, bat *batch.Batch, attrKeys []string, getter DataGetter, rowIdx int, typs []types.Type, mp *mpool.MPool) error {
   520  	var buf bytes.Buffer
   521  
   522  	for colIdx, typ := range typs {
   523  		id := typ.Oid
   524  		vec := bat.Vecs[colIdx]
   525  		fieldValue, ok := getter.GetFieldValue(attrKeys[colIdx])
   526  		if !ok || fieldValue == nil {
   527  			nulls.Add(vec.GetNulls(), uint64(rowIdx))
   528  			continue
   529  		}
   530  		switch id {
   531  		case types.T_bool:
   532  			var val bool
   533  			switch v := fieldValue.(type) {
   534  			case bool:
   535  				val = v
   536  			case int8, int16, int32, int64:
   537  				val = v != 0
   538  			case float32, float64:
   539  				val = v != 0.0
   540  			case string:
   541  				var err error
   542  				val, err = strconv.ParseBool(v)
   543  				if err != nil {
   544  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   545  					continue
   546  				}
   547  			default:
   548  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   549  				continue
   550  			}
   551  			cols := vector.MustFixedCol[bool](vec)
   552  			cols[rowIdx] = val
   553  		case types.T_bit:
   554  			switch v := fieldValue.(type) {
   555  			default:
   556  				strVal := fmt.Sprintf("%v", v)
   557  				val, err := strconv.ParseUint(strVal, 0, int(typ.Width))
   558  				if err != nil {
   559  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   560  					continue
   561  				}
   562  				cols := vector.MustFixedCol[uint64](vec)
   563  				cols[rowIdx] = val
   564  			}
   565  		case types.T_int8:
   566  			var val int8
   567  			switch v := fieldValue.(type) {
   568  			case float64:
   569  				if v < math.MinInt8 || v > math.MaxInt8 {
   570  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   571  					continue
   572  				}
   573  				val = int8(v)
   574  			case float32:
   575  				if v < math.MinInt8 || v > math.MaxInt8 {
   576  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   577  					continue
   578  				}
   579  				val = int8(v)
   580  			default:
   581  				strVal := fmt.Sprintf("%v", v)
   582  				parsedValue, err := strconv.ParseInt(strVal, 10, 8)
   583  				if err != nil {
   584  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   585  					continue
   586  				}
   587  				val = int8(parsedValue)
   588  			}
   589  			if err := vector.SetFixedAt(vec, rowIdx, val); err != nil {
   590  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   591  				continue
   592  			}
   593  		case types.T_int16:
   594  			var val int16
   595  			switch v := fieldValue.(type) {
   596  			case float64:
   597  				if v < math.MinInt16 || v > math.MaxInt16 {
   598  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   599  					continue
   600  				}
   601  				val = int16(v)
   602  			case float32:
   603  				if v < math.MinInt16 || v > math.MaxInt16 {
   604  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   605  					continue
   606  				}
   607  				val = int16(v)
   608  			default:
   609  				strVal := fmt.Sprintf("%v", v)
   610  				parsedValue, err := strconv.ParseInt(strVal, 10, 16)
   611  				if err != nil {
   612  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   613  					continue
   614  				}
   615  				val = int16(parsedValue)
   616  			}
   617  			if err := vector.SetFixedAt(vec, rowIdx, val); err != nil {
   618  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   619  				continue
   620  			}
   621  		case types.T_int32:
   622  			var val int32
   623  			switch v := fieldValue.(type) {
   624  			case float64:
   625  				if v < math.MinInt32 || v > math.MaxInt32 {
   626  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   627  					continue
   628  				}
   629  				val = int32(v)
   630  			case float32:
   631  				if v < math.MinInt32 || v > math.MaxInt32 {
   632  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   633  					continue
   634  				}
   635  				val = int32(v)
   636  			default:
   637  				strVal := fmt.Sprintf("%v", v)
   638  				parsedValue, err := strconv.ParseInt(strVal, 10, 32)
   639  				if err != nil {
   640  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   641  					continue
   642  				}
   643  				val = int32(parsedValue)
   644  			}
   645  			cols := vector.MustFixedCol[int32](vec)
   646  			cols[rowIdx] = val
   647  		case types.T_int64:
   648  			var val int64
   649  			var strVal string
   650  			switch v := fieldValue.(type) {
   651  			case float64:
   652  				if v < math.MinInt64 || v > math.MaxInt64 {
   653  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   654  					continue
   655  				}
   656  				val = int64(v)
   657  			case float32:
   658  				if v < math.MinInt64 || v > math.MaxInt64 {
   659  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   660  					continue
   661  				}
   662  				val = int64(v)
   663  			default:
   664  				strVal = fmt.Sprintf("%v", v)
   665  				parsedValue, err := strconv.ParseInt(strVal, 10, 64)
   666  				if err != nil {
   667  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   668  					continue
   669  				}
   670  				val = parsedValue
   671  			}
   672  			cols := vector.MustFixedCol[int64](vec)
   673  			cols[rowIdx] = val
   674  		case types.T_uint8:
   675  			var val uint8
   676  			switch v := fieldValue.(type) {
   677  			case float64:
   678  				if v < 0 || v > math.MaxUint8 {
   679  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   680  					continue
   681  				}
   682  				val = uint8(v)
   683  			case float32:
   684  				if v < 0 || v > math.MaxUint8 {
   685  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   686  					continue
   687  				}
   688  				val = uint8(v)
   689  			default:
   690  				strVal := fmt.Sprintf("%v", v)
   691  				parsedValue, err := strconv.ParseUint(strVal, 10, 8)
   692  				if err != nil {
   693  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   694  					continue
   695  				}
   696  				val = uint8(parsedValue)
   697  			}
   698  			cols := vector.MustFixedCol[uint8](vec)
   699  			cols[rowIdx] = val
   700  		case types.T_uint16:
   701  			var val uint16
   702  			switch v := fieldValue.(type) {
   703  			case float64:
   704  				if v < 0 || v > math.MaxUint16 {
   705  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   706  					continue
   707  				}
   708  				val = uint16(v)
   709  			case float32:
   710  				if v < 0 || v > math.MaxUint16 {
   711  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   712  					continue
   713  				}
   714  				val = uint16(v)
   715  			default:
   716  				strVal := fmt.Sprintf("%v", v)
   717  				parsedValue, err := strconv.ParseUint(strVal, 10, 16)
   718  				if err != nil {
   719  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   720  					continue
   721  				}
   722  				val = uint16(parsedValue)
   723  			}
   724  			cols := vector.MustFixedCol[uint16](vec)
   725  			cols[rowIdx] = val
   726  		case types.T_uint32:
   727  			var val uint32
   728  			switch v := fieldValue.(type) {
   729  			case float64:
   730  				if v < 0 || v > math.MaxUint32 {
   731  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   732  					continue
   733  				}
   734  				val = uint32(v)
   735  			case float32:
   736  				if v < 0 || v > math.MaxUint32 {
   737  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   738  					continue
   739  				}
   740  				val = uint32(v)
   741  			default:
   742  				strVal := fmt.Sprintf("%v", v)
   743  				parsedValue, err := strconv.ParseUint(strVal, 10, 32)
   744  				if err != nil {
   745  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   746  					continue
   747  				}
   748  				val = uint32(parsedValue)
   749  			}
   750  			cols := vector.MustFixedCol[uint32](vec)
   751  			cols[rowIdx] = val
   752  		case types.T_uint64:
   753  			var val uint64
   754  			switch v := fieldValue.(type) {
   755  			case float64:
   756  				if v < 0 || v > math.MaxUint64 {
   757  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   758  					continue
   759  				}
   760  				val = uint64(v)
   761  			case float32:
   762  				if v < 0 || v > math.MaxUint64 {
   763  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   764  					continue
   765  				}
   766  				val = uint64(v)
   767  			default:
   768  				strVal := fmt.Sprintf("%v", v)
   769  				parsedValue, err := strconv.ParseUint(strVal, 10, 64)
   770  				if err != nil {
   771  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   772  					continue
   773  				}
   774  				val = uint64(parsedValue)
   775  			}
   776  			cols := vector.MustFixedCol[uint64](vec)
   777  			cols[rowIdx] = val
   778  		case types.T_float32:
   779  			var val float32
   780  
   781  			switch v := fieldValue.(type) {
   782  			case float64:
   783  				if v < -math.MaxFloat32 || v > math.MaxFloat32 {
   784  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   785  					continue
   786  				}
   787  				val = float32(v)
   788  			case float32:
   789  				val = float32(v)
   790  			default:
   791  				strVal := fmt.Sprintf("%v", v)
   792  				parsedValue, err := strconv.ParseFloat(strVal, 32)
   793  				if err != nil {
   794  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   795  					continue
   796  				}
   797  				val = float32(parsedValue)
   798  			}
   799  			cols := vector.MustFixedCol[float32](vec)
   800  			cols[rowIdx] = val
   801  		case types.T_float64:
   802  			var val float64
   803  
   804  			switch v := fieldValue.(type) {
   805  			case float64:
   806  				val = float64(v)
   807  			case float32:
   808  				val = float64(v)
   809  			default:
   810  				strVal := fmt.Sprintf("%v", v)
   811  				parsedValue, err := strconv.ParseFloat(strVal, 32)
   812  				if err != nil {
   813  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   814  					continue
   815  				}
   816  				val = float64(parsedValue)
   817  			}
   818  			cols := vector.MustFixedCol[float64](vec)
   819  			cols[rowIdx] = val
   820  		case types.T_char, types.T_varchar, types.T_binary, types.T_varbinary, types.T_blob, types.T_text:
   821  			var strVal string
   822  			strVal = fmt.Sprintf("%v", fieldValue)
   823  			buf.WriteString(strVal)
   824  			bs := buf.Bytes()
   825  			err := vector.SetBytesAt(vec, rowIdx, bs, mp)
   826  			if err != nil {
   827  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   828  				continue
   829  			}
   830  			buf.Reset()
   831  		case types.T_json:
   832  			var jsonBytes []byte
   833  			valueStr := fmt.Sprintf("%v", fieldValue)
   834  			byteJson, err := types.ParseStringToByteJson(valueStr)
   835  			if err != nil {
   836  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   837  				continue
   838  			}
   839  			jsonBytes, err = types.EncodeJson(byteJson)
   840  			if err != nil {
   841  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   842  				continue
   843  			}
   844  			err = vector.SetBytesAt(vec, rowIdx, jsonBytes, mp)
   845  			if err != nil {
   846  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   847  				continue
   848  			}
   849  		case types.T_date:
   850  			valueStr := fmt.Sprintf("%v", fieldValue)
   851  			d, err := types.ParseDateCast(valueStr)
   852  			if err != nil {
   853  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   854  				continue
   855  			}
   856  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   857  				return err
   858  			}
   859  		case types.T_time:
   860  			valueStr := fmt.Sprintf("%v", fieldValue)
   861  			d, err := types.ParseTime(valueStr, vec.GetType().Scale)
   862  			if err != nil {
   863  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   864  				continue
   865  			}
   866  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   867  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   868  				continue
   869  			}
   870  		case types.T_timestamp:
   871  			valueStr := fmt.Sprintf("%v", fieldValue)
   872  			t := time.Local
   873  			d, err := types.ParseTimestamp(t, valueStr, vec.GetType().Scale)
   874  			if err != nil {
   875  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   876  				continue
   877  			}
   878  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   879  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   880  				continue
   881  			}
   882  		case types.T_datetime:
   883  			valueStr := fmt.Sprintf("%v", fieldValue)
   884  			d, err := types.ParseDatetime(valueStr, vec.GetType().Scale)
   885  			if err != nil {
   886  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   887  				continue
   888  			}
   889  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   890  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   891  				continue
   892  			}
   893  		case types.T_enum:
   894  			valueStr := fmt.Sprintf("%v", fieldValue)
   895  
   896  			d, err := strconv.ParseUint(valueStr, 10, 16)
   897  			if err == nil {
   898  				if err := vector.SetFixedAt(vec, rowIdx, uint16(d)); err != nil {
   899  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   900  					continue
   901  				}
   902  			} else {
   903  				if errors.Is(err, strconv.ErrRange) {
   904  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   905  					continue
   906  				}
   907  				f, err := strconv.ParseFloat(valueStr, 64)
   908  				if err != nil || f < 0 || f > math.MaxUint16 {
   909  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   910  					continue
   911  				}
   912  				if err := vector.SetFixedAt(vec, rowIdx, uint16(f)); err != nil {
   913  					return err
   914  				}
   915  			}
   916  		case types.T_decimal64:
   917  			valueStr := fmt.Sprintf("%v", fieldValue)
   918  
   919  			d, err := types.ParseDecimal64(valueStr, vec.GetType().Width, vec.GetType().Scale)
   920  			if err != nil {
   921  				if !moerr.IsMoErrCode(err, moerr.ErrDataTruncated) {
   922  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   923  					continue
   924  				}
   925  			}
   926  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   927  				return err
   928  			}
   929  		case types.T_decimal128:
   930  			valueStr := fmt.Sprintf("%v", fieldValue)
   931  			d, err := types.ParseDecimal128(valueStr, vec.GetType().Width, vec.GetType().Scale)
   932  			if err != nil {
   933  				// we tolerate loss of digits.
   934  				if !moerr.IsMoErrCode(err, moerr.ErrDataTruncated) {
   935  					nulls.Add(vec.GetNulls(), uint64(rowIdx))
   936  					continue
   937  				}
   938  			}
   939  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   940  				return err
   941  			}
   942  		case types.T_uuid:
   943  
   944  			valueStr := fmt.Sprintf("%v", fieldValue)
   945  
   946  			d, err := types.ParseUuid(valueStr)
   947  			if err != nil {
   948  				nulls.Add(vec.GetNulls(), uint64(rowIdx))
   949  				continue
   950  			}
   951  			if err := vector.SetFixedAt(vec, rowIdx, d); err != nil {
   952  				return err
   953  			}
   954  		default:
   955  			nulls.Add(vec.GetNulls(), uint64(rowIdx))
   956  			continue
   957  		}
   958  	}
   959  	return nil
   960  }
   961  
   962  func convertProtobufSchemaToMD(schema string, msgTypeName string) (*desc.MessageDescriptor, error) {
   963  	files := map[string]string{
   964  		"test.proto": schema,
   965  	}
   966  
   967  	parser := protoparse.Parser{
   968  		Accessor: protoparse.FileContentsFromMap(files),
   969  	}
   970  	fds, err := parser.ParseFiles("test.proto")
   971  
   972  	if err != nil {
   973  		return nil, err
   974  	}
   975  	fd := fds[0]
   976  	md := fd.FindMessage(msgTypeName)
   977  	return md, nil
   978  }
   979  
   980  func deserializeProtobuf(md *desc.MessageDescriptor, in []byte, isKafkSR bool) (*dynamic.Message, error) {
   981  	dm := dynamic.NewMessage(md)
   982  	var err error
   983  	if isKafkSR {
   984  		bytesRead, _, err := readMessageIndexes(in[5:])
   985  		if err != nil {
   986  			return nil, err
   987  		}
   988  		proto.Unmarshal(in[5+bytesRead:], dm)
   989  	} else {
   990  		err = dm.Unmarshal(in)
   991  	}
   992  	return dm, err
   993  }
   994  
   995  func readMessageIndexes(payload []byte) (int, []int, error) {
   996  	arrayLen, bytesRead := binary.Varint(payload)
   997  	if bytesRead <= 0 {
   998  		return bytesRead, nil, moerr.NewInternalError(context.Background(), "unable to read message indexes")
   999  	}
  1000  	if arrayLen == 0 {
  1001  		// Handle the optimization for the first message in the schema
  1002  		return bytesRead, []int{0}, nil
  1003  	}
  1004  	msgIndexes := make([]int, arrayLen)
  1005  	for i := 0; i < int(arrayLen); i++ {
  1006  		idx, read := binary.Varint(payload[bytesRead:])
  1007  		if read <= 0 {
  1008  			return bytesRead, nil, moerr.NewInternalError(context.Background(), "unable to read message indexes")
  1009  		}
  1010  		bytesRead += read
  1011  		msgIndexes[i] = int(idx)
  1012  	}
  1013  	return bytesRead, msgIndexes, nil
  1014  }
  1015  
  1016  func convertToKafkaConfig(configs map[string]interface{}) *kafka.ConfigMap {
  1017  	kafkaConfigs := &kafka.ConfigMap{}
  1018  	allowedKeys := map[string]struct{}{
  1019  		"bootstrap.servers": {},
  1020  		"security.protocol": {},
  1021  		"sasl.mechanisms":   {},
  1022  		"sasl.username":     {},
  1023  		"sasl.password":     {},
  1024  		// Add other Kafka-specific properties here...
  1025  	}
  1026  
  1027  	for key, value := range configs {
  1028  		if _, ok := allowedKeys[key]; ok {
  1029  			kafkaConfigs.SetKey(key, value)
  1030  		}
  1031  	}
  1032  	// each time we create a new consumer group for gather all messages
  1033  	groupId, _ := uuid.NewV7()
  1034  	kafkaConfigs.SetKey("group.id", groupId.String())
  1035  
  1036  	return kafkaConfigs
  1037  }
  1038  
  1039  func ValidateConfig(ctx context.Context, configs map[string]interface{}, factory func(configMap *kafka.ConfigMap) (KafkaAdapterInterface, error)) error {
  1040  	var requiredKeys = []string{
  1041  		TypeKey,
  1042  		TopicKey,
  1043  		ValueKey,
  1044  		BootstrapServersKey,
  1045  	}
  1046  
  1047  	var additionalAllowedKeys = []string{
  1048  		PartitionKey,
  1049  		RelkindKey,
  1050  		ProtobufMessagekey,
  1051  		ProtobufSchemaKey,
  1052  	}
  1053  
  1054  	// Create a set of allowed keys
  1055  	allowedKeys := make(map[string]struct{})
  1056  	for _, key := range requiredKeys {
  1057  		allowedKeys[key] = struct{}{}
  1058  	}
  1059  	for _, key := range additionalAllowedKeys {
  1060  		allowedKeys[key] = struct{}{}
  1061  	}
  1062  
  1063  	for _, key := range requiredKeys {
  1064  		if _, exists := configs[key]; !exists {
  1065  			return moerr.NewInternalError(ctx, "missing required key: %s", key)
  1066  		}
  1067  	}
  1068  
  1069  	// Validate keys in configs
  1070  	for key := range configs {
  1071  		if _, ok := allowedKeys[key]; !ok {
  1072  			return moerr.NewInternalError(ctx, "invalid key: %s", key)
  1073  		}
  1074  	}
  1075  
  1076  	value, ok := configs[ValueKey].(string)
  1077  	if !ok {
  1078  		return moerr.NewInternalError(ctx, "expected string value for key: %s", ValueKey)
  1079  	}
  1080  
  1081  	switch ValueType(value) {
  1082  	case JSON:
  1083  		// no additional checks required
  1084  	case PROTOBUF:
  1085  		// check the schema and message name has been set or not
  1086  		if _, ok := configs[ProtobufSchemaKey]; !ok {
  1087  			return moerr.NewInternalError(ctx, "missing required key: %s", ProtobufSchemaKey)
  1088  		}
  1089  		if _, ok := configs[ProtobufMessagekey]; !ok {
  1090  			return moerr.NewInternalError(ctx, "missing required key: %s", ProtobufMessagekey)
  1091  		}
  1092  	case PROTOBUFSR:
  1093  		if _, ok := configs[ProtobufMessagekey]; !ok {
  1094  			return moerr.NewInternalError(ctx, "missing required key: %s", ProtobufMessagekey)
  1095  		}
  1096  		if _, ok := configs[SchemaRegistryKey]; !ok {
  1097  			return moerr.NewInternalError(ctx, "missing required key: %s", SchemaRegistryKey)
  1098  		}
  1099  	default:
  1100  		return moerr.NewInternalError(ctx, "Unsupported value for key: %s", ValueKey)
  1101  	}
  1102  	// Convert the configuration to map[string]string for Kafka
  1103  	kafkaConfigs := convertToKafkaConfig(configs)
  1104  
  1105  	// Create the Kafka adapter
  1106  	ka, err := factory(kafkaConfigs)
  1107  	if err != nil {
  1108  		return err
  1109  	}
  1110  	defer ka.Close()
  1111  
  1112  	// Check if Topic exists
  1113  	_, err = ka.DescribeTopicDetails(ctx, configs[TopicKey].(string))
  1114  	if err != nil {
  1115  		return err
  1116  	}
  1117  	return nil
  1118  }
  1119  
  1120  type KafkaAdapterFactory func(configMap *kafka.ConfigMap) (KafkaAdapterInterface, error)
  1121  
  1122  func GetStreamCurrentSize(ctx context.Context, configs map[string]interface{}, factory KafkaAdapterFactory) (int64, error) {
  1123  	err := ValidateConfig(ctx, configs, NewKafkaAdapter)
  1124  	if err != nil {
  1125  		return 0, err
  1126  	}
  1127  
  1128  	configMap := convertToKafkaConfig(configs)
  1129  
  1130  	ka, err := factory(configMap)
  1131  	if err != nil {
  1132  		return 0, err
  1133  	}
  1134  	defer ka.Close()
  1135  
  1136  	meta, err := ka.DescribeTopicDetails(ctx, configs[TopicKey].(string))
  1137  	if err != nil {
  1138  		return 0, err
  1139  	}
  1140  
  1141  	var totalSize int64
  1142  	kaConsumer, _ := ka.GetKafkaConsumer()
  1143  
  1144  	var partitions []kafka.PartitionMetadata
  1145  	if configs[PartitionKey] != nil {
  1146  		partition, err := strconv.Atoi(configs[PartitionKey].(string))
  1147  		if err != nil {
  1148  			return 0, err
  1149  		}
  1150  		for _, p := range meta.Partitions {
  1151  			if p.ID == int32(partition) {
  1152  				partitions = append(partitions, p)
  1153  				break
  1154  			}
  1155  		}
  1156  	} else {
  1157  		partitions = meta.Partitions
  1158  	}
  1159  	for _, p := range partitions {
  1160  		// Fetch the high watermark for the partition
  1161  		_, highwatermarkHigh, err := kaConsumer.QueryWatermarkOffsets(configs[TopicKey].(string), p.ID, -1)
  1162  		if err != nil {
  1163  			return 0, err
  1164  		}
  1165  		totalSize += int64(highwatermarkHigh)
  1166  	}
  1167  	return totalSize, nil
  1168  }
  1169  
  1170  func RetrieveData(ctx context.Context, msgs []*kafka.Message, configs map[string]interface{}, attrs []string, types []types.Type, offset int64, limit int64, mp *mpool.MPool, factory KafkaAdapterFactory) (*batch.Batch, error) {
  1171  	err := ValidateConfig(ctx, configs, NewKafkaAdapter)
  1172  	if err != nil {
  1173  		return nil, err
  1174  	}
  1175  
  1176  	configMap := convertToKafkaConfig(configs)
  1177  
  1178  	ka, err := factory(configMap)
  1179  	if err != nil {
  1180  		return nil, err
  1181  	}
  1182  	defer ka.Close()
  1183  
  1184  	// init schema registry client if schema registry url is set
  1185  	if sr, ok := configs[SchemaRegistryKey]; ok {
  1186  		err = ka.InitSchemaRegistry(sr.(string))
  1187  		if err != nil {
  1188  			return nil, err
  1189  		}
  1190  	}
  1191  
  1192  	var messages []*kafka.Message // Replace 'YourMessageType' with the actual type of your messages
  1193  
  1194  	// Determine the source of messages based on whether 'msgs' is nil or not
  1195  	if msgs != nil {
  1196  		messages = msgs
  1197  	} else {
  1198  		var err error
  1199  		messages, err = ka.ReadMessagesFromTopic(configs[TopicKey].(string), offset, limit, configs)
  1200  		if err != nil {
  1201  			return nil, err
  1202  		}
  1203  	}
  1204  
  1205  	// Common logic for processing messages
  1206  	b, err := PopulateBatchFromMSG(ctx, ka, types, attrs, messages, configs, mp)
  1207  	if err != nil {
  1208  		return nil, err
  1209  	}
  1210  
  1211  	return b, nil
  1212  
  1213  }