github.com/confluentinc/confluent-kafka-go@v1.9.2/schemaregistry/serde/serde.go (about) 1 /** 2 * Copyright 2022 Confluent Inc. 3 * 4 * Licensed under the Apache License, Version 2.0 (the "License"); 5 * you may not use this file except in compliance with the License. 6 * You may obtain a copy of the License at 7 * 8 * http://www.apache.org/licenses/LICENSE-2.0 9 * 10 * Unless required by applicable law or agreed to in writing, software 11 * distributed under the License is distributed on an "AS IS" BASIS, 12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 13 * See the License for the specific language governing permissions and 14 * limitations under the License. 15 */ 16 17 package serde 18 19 import "C" 20 import ( 21 "bytes" 22 "encoding/binary" 23 "fmt" 24 25 "github.com/confluentinc/confluent-kafka-go/schemaregistry" 26 ) 27 28 // Type represents the type of Serde 29 type Type = int 30 31 const ( 32 // KeySerde denotes a key Serde 33 KeySerde = 1 34 // ValueSerde denotes a value Serde 35 ValueSerde = 2 36 ) 37 38 const ( 39 // EnableValidation enables validation 40 EnableValidation = true 41 // DisableValidation disables validation 42 DisableValidation = false 43 ) 44 45 // magicByte is prepended to the serialized payload 46 const magicByte byte = 0x0 47 48 // MessageFactory is a factory function, which should return a pointer to 49 // an instance into which we will unmarshal wire data. 50 // For Avro, the name will be the name of the Avro type if it has one. 51 // For JSON Schema, the name will be empty. 52 // For Protobuf, the name will be the name of the message type. 53 type MessageFactory func(subject string, name string) (interface{}, error) 54 55 // Serializer represents a serializer 56 type Serializer interface { 57 ConfigureSerializer(client schemaregistry.Client, serdeType Type, conf *SerializerConfig) error 58 // Serialize will serialize the given message, which should be a pointer. 59 // For example, in Protobuf, messages are always a pointer to a struct and never just a struct. 60 Serialize(topic string, msg interface{}) ([]byte, error) 61 Close() 62 } 63 64 // Deserializer represents a deserializer 65 type Deserializer interface { 66 ConfigureDeserializer(client schemaregistry.Client, serdeType Type, conf *DeserializerConfig) error 67 // Deserialize will call the MessageFactory to create an object 68 // into which we will unmarshal data. 69 Deserialize(topic string, payload []byte) (interface{}, error) 70 // DeserializeInto will unmarshal data into the given object. 71 DeserializeInto(topic string, payload []byte, msg interface{}) error 72 Close() 73 } 74 75 // Serde is a common instance for both the serializers and deserializers 76 type Serde struct { 77 Client schemaregistry.Client 78 SerdeType Type 79 SubjectNameStrategy SubjectNameStrategyFunc 80 } 81 82 // BaseSerializer represents basic serializer info 83 type BaseSerializer struct { 84 Serde 85 Conf *SerializerConfig 86 } 87 88 // BaseDeserializer represents basic deserializer info 89 type BaseDeserializer struct { 90 Serde 91 Conf *DeserializerConfig 92 MessageFactory MessageFactory 93 } 94 95 // ConfigureSerializer configures the Serializer 96 func (s *BaseSerializer) ConfigureSerializer(client schemaregistry.Client, serdeType Type, conf *SerializerConfig) error { 97 if client == nil { 98 return fmt.Errorf("schema registry client missing") 99 } 100 s.Client = client 101 s.Conf = conf 102 s.SerdeType = serdeType 103 s.SubjectNameStrategy = TopicNameStrategy 104 return nil 105 } 106 107 // ConfigureDeserializer configures the Deserializer 108 func (s *BaseDeserializer) ConfigureDeserializer(client schemaregistry.Client, serdeType Type, conf *DeserializerConfig) error { 109 if client == nil { 110 return fmt.Errorf("schema registry client missing") 111 } 112 s.Client = client 113 s.Conf = conf 114 s.SerdeType = serdeType 115 s.SubjectNameStrategy = TopicNameStrategy 116 return nil 117 } 118 119 // SubjectNameStrategyFunc determines the subject for the given parameters 120 type SubjectNameStrategyFunc func(topic string, serdeType Type, schema schemaregistry.SchemaInfo) (string, error) 121 122 // TopicNameStrategy creates a subject name by appending -[key|value] to the topic name. 123 func TopicNameStrategy(topic string, serdeType Type, schema schemaregistry.SchemaInfo) (string, error) { 124 suffix := "-value" 125 if serdeType == KeySerde { 126 suffix = "-key" 127 } 128 return topic + suffix, nil 129 } 130 131 // GetID returns a schema ID for the given schema 132 func (s *BaseSerializer) GetID(topic string, msg interface{}, info schemaregistry.SchemaInfo) (int, error) { 133 autoRegister := s.Conf.AutoRegisterSchemas 134 useSchemaID := s.Conf.UseSchemaID 135 useLatest := s.Conf.UseLatestVersion 136 normalizeSchema := s.Conf.NormalizeSchemas 137 138 var id = -1 139 subject, err := s.SubjectNameStrategy(topic, s.SerdeType, info) 140 if err != nil { 141 return -1, err 142 } 143 if autoRegister { 144 id, err = s.Client.Register(subject, info, normalizeSchema) 145 if err != nil { 146 return -1, err 147 } 148 } else if useSchemaID >= 0 { 149 info, err = s.Client.GetBySubjectAndID(subject, useSchemaID) 150 if err != nil { 151 return -1, err 152 } 153 _, err := s.Client.GetID(subject, info, false) 154 if err != nil { 155 return -1, err 156 } 157 } else if useLatest { 158 metadata, err := s.Client.GetLatestSchemaMetadata(subject) 159 if err != nil { 160 return -1, err 161 } 162 info = schemaregistry.SchemaInfo{ 163 Schema: metadata.Schema, 164 SchemaType: metadata.SchemaType, 165 References: metadata.References, 166 } 167 id, err = s.Client.GetID(subject, info, false) 168 if err != nil { 169 return -1, err 170 } 171 } else { 172 id, err = s.Client.GetID(subject, info, normalizeSchema) 173 if err != nil { 174 return -1, err 175 } 176 } 177 return id, nil 178 } 179 180 // WriteBytes writes the serialized payload prepended by the magicByte 181 func (s *BaseSerializer) WriteBytes(id int, msgBytes []byte) ([]byte, error) { 182 var buf bytes.Buffer 183 err := buf.WriteByte(magicByte) 184 if err != nil { 185 return nil, err 186 } 187 idBytes := make([]byte, 4) 188 binary.BigEndian.PutUint32(idBytes, uint32(id)) 189 _, err = buf.Write(idBytes) 190 if err != nil { 191 return nil, err 192 } 193 _, err = buf.Write(msgBytes) 194 if err != nil { 195 return nil, err 196 } 197 return buf.Bytes(), nil 198 } 199 200 // GetSchema returns a schema for a payload 201 func (s *BaseDeserializer) GetSchema(topic string, payload []byte) (schemaregistry.SchemaInfo, error) { 202 info := schemaregistry.SchemaInfo{} 203 if payload[0] != magicByte { 204 return info, fmt.Errorf("unknown magic byte") 205 } 206 id := binary.BigEndian.Uint32(payload[1:5]) 207 subject, err := s.SubjectNameStrategy(topic, s.SerdeType, info) 208 if err != nil { 209 return info, err 210 } 211 return s.Client.GetBySubjectAndID(subject, int(id)) 212 } 213 214 // ResolveReferences resolves schema references 215 func ResolveReferences(c schemaregistry.Client, schema schemaregistry.SchemaInfo, deps map[string]string) error { 216 for _, ref := range schema.References { 217 metadata, err := c.GetSchemaMetadata(ref.Subject, ref.Version) 218 if err != nil { 219 return err 220 } 221 info := schemaregistry.SchemaInfo{ 222 Schema: metadata.Schema, 223 SchemaType: metadata.SchemaType, 224 References: metadata.References, 225 } 226 deps[ref.Name] = metadata.Schema 227 err = ResolveReferences(c, info, deps) 228 if err != nil { 229 return err 230 } 231 } 232 return nil 233 } 234 235 // Close closes the Serde 236 func (s *Serde) Close() { 237 }