github.com/matrixorigin/matrixone@v1.2.0/pkg/stream/adapter/kafka/adapter_test.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  	"context"
    19  	"encoding/json"
    20  	"fmt"
    21  	"os"
    22  	"strings"
    23  	"testing"
    24  
    25  	"github.com/confluentinc/confluent-kafka-go/v2/schemaregistry"
    26  	"github.com/matrixorigin/matrixone/pkg/container/vector"
    27  
    28  	"github.com/confluentinc/confluent-kafka-go/v2/kafka"
    29  	"github.com/gogo/protobuf/proto"
    30  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    31  	"github.com/matrixorigin/matrixone/pkg/container/types"
    32  	"github.com/matrixorigin/matrixone/pkg/stream/adapter/kafka/test/proto/test_v1"
    33  	"github.com/stretchr/testify/assert"
    34  )
    35  
    36  type MockKafkaAdapter struct{}
    37  
    38  func (m *MockKafkaAdapter) InitSchemaRegistry(url string) error {
    39  	return nil // Mocked response
    40  }
    41  
    42  func (m *MockKafkaAdapter) Close() {
    43  	// Mocked response
    44  }
    45  
    46  func (m *MockKafkaAdapter) GetKafkaConsumer() (*kafka.Consumer, error) {
    47  	return nil, nil // Mocked response
    48  }
    49  func (m *MockKafkaAdapter) CreateTopic(ctx context.Context, topicName string, partitions int, replicationFactor int) error {
    50  	return nil // Mocked response
    51  }
    52  
    53  func (m *MockKafkaAdapter) DescribeTopicDetails(ctx context.Context, topicName string) (*kafka.TopicMetadata, error) {
    54  	return nil, nil // Mocked response
    55  }
    56  
    57  func (m *MockKafkaAdapter) ReadMessagesFromPartition(topic string, partition int32, offset int64, limit int) ([]*kafka.Message, error) {
    58  	return nil, nil // Mocked response
    59  }
    60  
    61  func (m *MockKafkaAdapter) ReadMessagesFromTopic(topic string, offset int64, limit int64, configs map[string]interface{}) ([]*kafka.Message, error) {
    62  	return nil, nil // Mocked response
    63  }
    64  
    65  func (m *MockKafkaAdapter) GetSchemaForTopic(topic string, isKey bool) (schemaregistry.SchemaMetadata, error) {
    66  	return schemaregistry.SchemaMetadata{}, nil // Mocked response
    67  }
    68  
    69  func (m *MockKafkaAdapter) ProduceMessage(topic string, key, value []byte) (int64, error) {
    70  	return 0, nil // Mocked response
    71  }
    72  
    73  func TestNewKafkaAdapter(t *testing.T) {
    74  
    75  	mockCluster, err := kafka.NewMockCluster(1)
    76  	if err != nil {
    77  		fmt.Printf("Failed to create MockCluster: %s\n", err)
    78  		os.Exit(1)
    79  	}
    80  	defer mockCluster.Close()
    81  
    82  	broker := mockCluster.BootstrapServers()
    83  
    84  	p, err := kafka.NewProducer(&kafka.ConfigMap{"bootstrap.servers": broker})
    85  
    86  	if err != nil {
    87  		fmt.Printf("Failed to create producer: %s\n", err)
    88  		os.Exit(1)
    89  	}
    90  
    91  	fmt.Printf("Created Producer %v\n", p)
    92  	deliveryChan := make(chan kafka.Event)
    93  
    94  	topic := "Test"
    95  	value := "Hello Go!"
    96  	p.Produce(&kafka.Message{
    97  		TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: kafka.PartitionAny},
    98  		Value:          []byte(value),
    99  		Headers:        []kafka.Header{{Key: "myTestHeader", Value: []byte("header values are binary")}},
   100  	}, deliveryChan)
   101  
   102  	e := <-deliveryChan
   103  	m := e.(*kafka.Message)
   104  
   105  	if m.TopicPartition.Error != nil {
   106  		fmt.Printf("Delivery failed: %v\n", m.TopicPartition.Error)
   107  	} else {
   108  		fmt.Printf("Delivered message to topic %s [%d] at offset %v\n",
   109  			*m.TopicPartition.Topic, m.TopicPartition.Partition, m.TopicPartition.Offset)
   110  	}
   111  
   112  	close(deliveryChan)
   113  
   114  	c, err := kafka.NewConsumer(&kafka.ConfigMap{
   115  		"bootstrap.servers":     broker,
   116  		"broker.address.family": "v4",
   117  		"group.id":              "group",
   118  		"session.timeout.ms":    6000,
   119  		"auto.offset.reset":     "earliest"})
   120  
   121  	if err != nil {
   122  		fmt.Fprintf(os.Stderr, "Failed to create consumer: %s\n", err)
   123  		os.Exit(1)
   124  	}
   125  	defer c.Close()
   126  
   127  	fmt.Printf("Created Consumer %v\n", c)
   128  
   129  	err = c.SubscribeTopics([]string{topic}, nil)
   130  	if err != nil {
   131  		fmt.Fprintf(os.Stderr, "Failed to subscribe to consumer: %s\n", err)
   132  		os.Exit(1)
   133  	}
   134  
   135  	msg, err := c.ReadMessage(-1)
   136  	if err != nil {
   137  		fmt.Fprintf(os.Stderr, "Failed to read message: %s\n", err)
   138  		os.Exit(1)
   139  	}
   140  
   141  	fmt.Println("received message: ", string(msg.Value))
   142  
   143  }
   144  
   145  func TestValidateConfig_MissingRequiredKeys(t *testing.T) {
   146  	ctx := context.Background()
   147  	configs := map[string]interface{}{
   148  		// Intentionally leaving out some keys to simulate missing keys
   149  		TypeKey:             "someType",
   150  		BootstrapServersKey: "localhost:9092",
   151  	}
   152  
   153  	err := ValidateConfig(ctx, configs, NewKafkaAdapter)
   154  	if err == nil || !strings.Contains(err.Error(), "missing required key") {
   155  		t.Errorf("Expected an error about missing keys, got: %v", err)
   156  	}
   157  }
   158  
   159  func TestValidateConfig_UnsupportedValueKey(t *testing.T) {
   160  	ctx := context.Background()
   161  	configs := map[string]interface{}{
   162  		TypeKey:             "someType",
   163  		TopicKey:            "someTopic",
   164  		ValueKey:            "UNSUPPORTED_VALUE",
   165  		BootstrapServersKey: "localhost:9092",
   166  	}
   167  
   168  	err := ValidateConfig(ctx, configs, NewKafkaAdapter)
   169  	if err == nil || !strings.Contains(err.Error(), "Unsupported value for key") {
   170  		t.Errorf("Expected an error about unsupported value, got: %v", err)
   171  	}
   172  }
   173  
   174  func TestValidateConfig_ValidConfigurations(t *testing.T) {
   175  	ctx := context.Background()
   176  	configs := map[string]interface{}{
   177  		TypeKey:             "someType",
   178  		TopicKey:            "someTopic",
   179  		ValueKey:            "json", // or PROTOBUF
   180  		BootstrapServersKey: "localhost:9092",
   181  	}
   182  
   183  	// Assuming you have a way to stub out or mock NewKafkaAdapter and DescribeTopicDetails here
   184  	mockFactory := func(configMap *kafka.ConfigMap) (KafkaAdapterInterface, error) {
   185  		return &MockKafkaAdapter{}, nil
   186  	}
   187  	err := ValidateConfig(ctx, configs, mockFactory)
   188  	if err != nil {
   189  		t.Errorf("Did not expect an error, got: %v", err)
   190  	}
   191  }
   192  
   193  func TestRetrieveDataWIthJson(t *testing.T) {
   194  	// Setup
   195  	mockCluster, err := kafka.NewMockCluster(1)
   196  	if err != nil {
   197  		t.Fatalf("Failed to create MockCluster: %s", err)
   198  	}
   199  	defer mockCluster.Close()
   200  
   201  	broker := mockCluster.BootstrapServers()
   202  	topic := "TestTopic"
   203  
   204  	// Produce mock data
   205  	// (You can add more messages or customize this part as necessary)
   206  	p, err := kafka.NewProducer(&kafka.ConfigMap{"bootstrap.servers": broker})
   207  	if err != nil {
   208  		t.Fatalf("Failed to create producer: %s", err)
   209  	}
   210  	type MessagePayload struct {
   211  		Name string `json:"name"`
   212  		Age  int32  `json:"age"`
   213  	}
   214  	payload := MessagePayload{
   215  		Name: "test_name",
   216  		Age:  100,
   217  	}
   218  	value, _ := json.Marshal(payload)
   219  
   220  	// produce 100 messages
   221  	for i := 0; i < 100; i++ {
   222  		err := p.Produce(&kafka.Message{
   223  			TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: kafka.PartitionAny},
   224  			Value:          value,
   225  		}, nil)
   226  		if err != nil {
   227  			return
   228  		}
   229  	}
   230  
   231  	// Setup configs for RetrieveData
   232  	configs := map[string]interface{}{
   233  		"type":              "kafka",
   234  		"bootstrap.servers": broker,
   235  		"topic":             topic,
   236  		"value":             "json",
   237  	}
   238  	attrs := []string{
   239  		"name", "age",
   240  	}
   241  	types := []types.Type{
   242  		types.New(types.T_char, 30, 0),
   243  		types.New(types.T_int32, 10, 0),
   244  	}
   245  	offset := int64(0)
   246  	limit := int64(50)
   247  
   248  	// Call RetrieveData
   249  	batch, err := RetrieveData(context.Background(), nil, configs, attrs, types, offset, limit, mpool.MustNewZero(), NewKafkaAdapter)
   250  	if err != nil {
   251  		t.Fatalf("RetrieveData failed: %s", err)
   252  	}
   253  
   254  	// Assertions
   255  	assert.Equal(t, 2, batch.VectorCount(), "Expected 2 vectors in the batch")
   256  	assert.Equal(t, batch.Vecs[0].Length(), 50, "Expected 50 row in the batch")
   257  }
   258  func TestPopulateBatchFromMSGWithJSON(t *testing.T) {
   259  
   260  	// Define types and attributes as per your schema
   261  	typs := []types.Type{
   262  		types.New(types.T_char, 30, 0),
   263  		types.New(types.T_int32, 32, 0),
   264  		types.New(types.T_int16, 16, 0),
   265  		types.New(types.T_int8, 8, 0),
   266  		types.New(types.T_uint8, 8, 0),
   267  		types.New(types.T_uint64, 64, 0),
   268  		types.New(types.T_float32, 32, 0),
   269  		types.New(types.T_datetime, 64, 0),
   270  		types.New(types.T_json, 64, 0),
   271  	}
   272  	attrs := []string{
   273  		"1", "2", "3", "4", "5", "6", "7", "8", "9",
   274  	}
   275  	configs := map[string]interface{}{
   276  		"value": "json",
   277  	}
   278  	// Create test msgs data
   279  	msgs := []*kafka.Message{
   280  		{
   281  			Value: []byte(`{
   282              "1": "test_name1",
   283              "2": 99,
   284              "3": 123,
   285              "4": 1,
   286              "5": 2,
   287              "6": 1234567890123,
   288              "7": 123.3,
   289              "8": "2021-01-01",
   290              "9": "{\"a\": 1}"
   291          }`),
   292  		},
   293  		{
   294  			Value: []byte(`{
   295  				"1": "test_name2",
   296  				"2": 150,
   297  				"3": 456,
   298  				"4": 3,
   299  				"5": 4,
   300  				"6": 9876543210987,
   301  				"7": 456.7,
   302  				"8": "2000-12-04",
   303  				"9": "{\"c1\": \"\\\"\",\"c2\":\"\\\\:\",\"c3\": \"\\\\,\",\"c4\":\"\\\\\\\"\\\\:\\\\,\",\"c5\":\"\\\\{\",\"c6\":\"\\\\}\"}"
   304  		}
   305  		`),
   306  		},
   307  		{
   308  			Value: []byte(`{
   309              "1": "test_name3",
   310              "2": 2147483648,
   311              "3": 32768,
   312              "4": 128,
   313              "5": 256,
   314              "6": 1844674407370955161644444,
   315              "7": "123.33.3",
   316              "8": "2021-01-01T00:00:00",
   317              "9": "invalid_json"
   318          }`),
   319  		},
   320  	}
   321  
   322  	expected := [][]interface{}{
   323  		{
   324  			"test_name1",
   325  			int32(99),
   326  			int16(123),
   327  			int8(1),
   328  			uint8(2),
   329  			uint64(1234567890123),
   330  			float32(123.3),
   331  			types.Datetime(63745056000000000),
   332  			"\x01\x01\x00\x00\x00\x1c\x00\x00\x00\x13\x00\x00\x00\x01\x00\t\x14\x00\x00\x00a\x01\x00\x00\x00\x00\x00\x00\x00",
   333  		},
   334  		{
   335  			"test_name2",
   336  			int32(150),
   337  			int16(456),
   338  			int8(3),
   339  			uint8(4),
   340  			uint64(9876543210987),
   341  			float32(456.7),
   342  			types.Datetime(63111484800000000),
   343  			"\x01\x06\x00\x00\x00k\x00\x00\x00J\x00\x00\x00\x02\x00L\x00\x00\x00\x02\x00N\x00\x00\x00\x02\x00P\x00\x00\x00\x02\x00R\x00\x00\x00\x02\x00T\x00\x00\x00\x02\x00\fV\x00\x00\x00\fX\x00\x00\x00\f[\x00\x00\x00\f^\x00\x00\x00\fe\x00\x00\x00\fh\x00\x00\x00c1c2c3c4c5c6\x01\"\x02\\:\x02\\,\x06\\\"\\:\\,\x02\\{\x02\\}",
   344  		},
   345  		{
   346  			"test_name3",
   347  			nil, // int32 overflow
   348  			nil, // int16 overflow
   349  			nil, // int8 overflow
   350  			nil, // uint8 overflow
   351  			nil, // uint64 overflow
   352  			nil, // valid float32
   353  			nil, // invalid datetime
   354  			nil, // invalid JSON
   355  		},
   356  	}
   357  
   358  	// Call PopulateBatchFromMSG
   359  	batch, err := PopulateBatchFromMSG(context.Background(), nil, typs, attrs, msgs, configs, mpool.MustNewZero())
   360  	if err != nil {
   361  		t.Fatalf("PopulateBatchFromMSG failed: %s", err)
   362  	}
   363  
   364  	// Assertions
   365  	if batch == nil {
   366  		t.Errorf("Expected non-nil batch")
   367  	} else {
   368  
   369  		// Check the data in the batch
   370  		for colIdx, attr := range attrs {
   371  			vec := batch.Vecs[colIdx]
   372  			if vec.Length() != len(msgs) {
   373  				t.Errorf("Expected %d rows in column '%s', got %d", len(msgs), attr, vec.Length())
   374  			} else {
   375  				for rowIdx := 0; rowIdx < vec.Length(); rowIdx++ {
   376  					expectedValue := expected[rowIdx][colIdx]
   377  					actualValue := getNonNullValue(vec, uint32(rowIdx))
   378  					if vec.GetNulls().Contains(uint64(rowIdx)) {
   379  						actualValue = nil
   380  					}
   381  					assert.Equal(t, expectedValue, actualValue, fmt.Sprintf("Mismatch in row %d, column '%s'", rowIdx, attr))
   382  				}
   383  			}
   384  		}
   385  	}
   386  
   387  }
   388  
   389  func getNonNullValue(col *vector.Vector, row uint32) any {
   390  
   391  	switch col.GetType().Oid {
   392  	case types.T_bool:
   393  		return vector.GetFixedAt[bool](col, int(row))
   394  	case types.T_int8:
   395  		return vector.GetFixedAt[int8](col, int(row))
   396  	case types.T_int16:
   397  		return vector.GetFixedAt[int16](col, int(row))
   398  	case types.T_int32:
   399  		return vector.GetFixedAt[int32](col, int(row))
   400  	case types.T_int64:
   401  		return vector.GetFixedAt[int64](col, int(row))
   402  	case types.T_uint8:
   403  		return vector.GetFixedAt[uint8](col, int(row))
   404  	case types.T_uint16:
   405  		return vector.GetFixedAt[uint16](col, int(row))
   406  	case types.T_uint32:
   407  		return vector.GetFixedAt[uint32](col, int(row))
   408  	case types.T_uint64:
   409  		return vector.GetFixedAt[uint64](col, int(row))
   410  	case types.T_decimal64:
   411  		return vector.GetFixedAt[types.Decimal64](col, int(row))
   412  	case types.T_decimal128:
   413  		return vector.GetFixedAt[types.Decimal128](col, int(row))
   414  	case types.T_uuid:
   415  		return vector.GetFixedAt[types.Uuid](col, int(row))
   416  	case types.T_float32:
   417  		return vector.GetFixedAt[float32](col, int(row))
   418  	case types.T_float64:
   419  		return vector.GetFixedAt[float64](col, int(row))
   420  	case types.T_date:
   421  		return vector.GetFixedAt[types.Date](col, int(row))
   422  	case types.T_time:
   423  		return vector.GetFixedAt[types.Time](col, int(row))
   424  	case types.T_datetime:
   425  		return vector.GetFixedAt[types.Datetime](col, int(row))
   426  	case types.T_timestamp:
   427  		return vector.GetFixedAt[types.Timestamp](col, int(row))
   428  	case types.T_enum:
   429  		return vector.GetFixedAt[types.Enum](col, int(row))
   430  	case types.T_TS:
   431  		return vector.GetFixedAt[types.TS](col, int(row))
   432  	case types.T_Rowid:
   433  		return vector.GetFixedAt[types.Rowid](col, int(row))
   434  	case types.T_Blockid:
   435  		return vector.GetFixedAt[types.Blockid](col, int(row))
   436  	case types.T_json:
   437  		return col.GetStringAt(int(row))
   438  	case types.T_char, types.T_varchar, types.T_binary, types.T_varbinary, types.T_blob, types.T_text,
   439  		types.T_array_float32, types.T_array_float64:
   440  		return col.GetStringAt(int(row))
   441  	default:
   442  		//return vector.ErrVecTypeNotSupport
   443  		panic(any("No Support"))
   444  	}
   445  }
   446  
   447  func TestRetrieveDataWIthProtobuf(t *testing.T) {
   448  	// Setup
   449  	mockCluster, err := kafka.NewMockCluster(1)
   450  	if err != nil {
   451  		t.Fatalf("Failed to create MockCluster: %s", err)
   452  	}
   453  	defer mockCluster.Close()
   454  
   455  	broker := mockCluster.BootstrapServers()
   456  	topic := "TestTopic"
   457  
   458  	// Produce mock data
   459  	p, err := kafka.NewProducer(&kafka.ConfigMap{"bootstrap.servers": broker})
   460  	if err != nil {
   461  		t.Fatalf("Failed to create producer: %s", err)
   462  	}
   463  	defer p.Close()
   464  
   465  	user := test_v1.UserMessage{
   466  		Name:  "dummy_name",
   467  		Age:   10,
   468  		Email: "dummy@test.com",
   469  	}
   470  	payload, _ := proto.Marshal(&user)
   471  
   472  	// produce 100 messages
   473  	for i := 0; i < 100; i++ {
   474  		p.Produce(&kafka.Message{
   475  			TopicPartition: kafka.TopicPartition{Topic: &topic, Partition: kafka.PartitionAny},
   476  			Value:          payload,
   477  		}, nil)
   478  	}
   479  
   480  	// Setup configs for RetrieveData
   481  	configs := map[string]interface{}{
   482  		"type":              "kafka",
   483  		"bootstrap.servers": broker,
   484  		"topic":             topic,
   485  		"value":             "protobuf",
   486  		"protobuf.message":  "test_v1.UserMessage",
   487  		"protobuf.schema":   "syntax = \"proto3\";\noption go_package = \"./proto/test_v1\";\npackage test_v1;\n\nmessage UserMessage {\n  string Name = 1;\n  string Email = 2;\n  int32 Age = 4;\n}",
   488  	}
   489  	attrs := []string{
   490  		"Name", "Age",
   491  	}
   492  	types := []types.Type{
   493  		types.New(types.T_char, 30, 0),
   494  		types.New(types.T_int32, 10, 0),
   495  	}
   496  	offset := int64(0)
   497  	limit := int64(50)
   498  
   499  	// Call RetrieveData
   500  	batch, err := RetrieveData(context.Background(), nil, configs, attrs, types, offset, limit, mpool.MustNewZero(), NewKafkaAdapter)
   501  	if err != nil {
   502  		t.Fatalf("RetrieveData failed: %s", err)
   503  	}
   504  
   505  	// Assertions
   506  	assert.Equal(t, 2, batch.VectorCount(), "Expected 2 vectors in the batch")
   507  	assert.Equal(t, batch.Vecs[0].Length(), 50, "Expected 50 row in the batch")
   508  
   509  }