You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2020/09/21 06:07:47 UTC

[GitHub] [pulsar-client-go] wolfstudy opened a new pull request #368: [WIP] Add schema logic for Go client

wolfstudy opened a new pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368


   Signed-off-by: xiaolong.ran <rx...@apache.org>
   
   
   
   Fixes #344 
   
   
   ### Motivation
   
   Type safety is extremely important in any application built around a message bus like Pulsar. Currently, Apache Pulsar supports the function of Schema Registry. And Java, CPP and Python clients already support schema registry related functions, In order to further improve the function of Go Client, we need to support the function of Schema Registry in Go Client.
   
   ### Modifications
   
   - Add schema logic in producer
   - Add schema logic in consumer
   - Add schema logic in reader
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] cckellogg commented on a change in pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
cckellogg commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r504847055



##########
File path: pulsar/schemaDef_test.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       we should be consistent with name. all the files are snake_case and I suggest we keep it that way for consistency. schema_def_test.go. Same for the other files in the PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503661148



##########
File path: pulsar/schema.go
##########
@@ -0,0 +1,502 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"reflect"
+	"unsafe"
+
+	log "github.com/sirupsen/logrus"
+
+	"github.com/gogo/protobuf/proto"
+	"github.com/linkedin/goavro"
+)
+
+type SchemaType int

Review comment:
       Maybe we can use SchemaInfo.Name replace it, right?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503657306



##########
File path: pulsar/schemaDef_test.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       Yes, this test case is just to test whether the logic of the schemaDef is correct. When we implement the schema function on the producer and consumer respectively, we will add separate test cases for each schema type. At this time, the file will be named `schema_test.go`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-714252295


   @skyrocknroll done PTAL again, thanks. 
   
   ping @sijie @massakam If time is convenient, please review it, thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy edited a comment on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
wolfstudy edited a comment on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-714250012


   > @wolfstudy is there any specific reason why we are using v1.0.5 of goavro ? According to README `"github.com/linkedin/goavro/v2"` has better support for namespaces and performance improvements. Can we please switch to v2 ?
   
   Cool, I think we can update the `gpavro` to `v2`, this looks good to me


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-714250012


   > @wolfstudy is there any specific reason why we are using v1.0.5 of goavro ? According to README `"github.com/linkedin/goavro/v2"` has better support for namespaces and performance improvements. Can we please switch to v2 ?
   
   Cool, I think we can update the `gpavro` to `v2`


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503657990



##########
File path: pulsar/message.go
##########
@@ -27,6 +27,9 @@ type ProducerMessage struct {
 	// Payload for the message
 	Payload []byte
 
+	//Value and payload is mutually exclusive, `Value interface{}` for schema message.
+	Value interface{}

Review comment:
       Should not be private, this interface is to be exposed to users, for example:
   
   ```
   err := producer.Send(context.Background(), ProducerMessage{
   		Value: &testJson{
   			ID:   100,
   			Name: "pulsar",
   		},
   	})
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] skyrocknroll commented on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
skyrocknroll commented on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-713612945


   @wolfstudy is there any specific reason we are using v1.0.5 of goavro. According to README `"github.com/linkedin/goavro/v2"` has better support for namespaces and performance improvements. Can we please switch to v2 ? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy merged pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
wolfstudy merged pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] skyrocknroll edited a comment on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
skyrocknroll edited a comment on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-713612945


   @wolfstudy is there any specific reason why we are using v1.0.5 of goavro ? According to README `"github.com/linkedin/goavro/v2"` has better support for namespaces and performance improvements. Can we please switch to v2 ? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] cckellogg commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
cckellogg commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503356983



##########
File path: pulsar/schemaDef_test.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one

Review comment:
       rename file schema_test.go. We should use snake_case to be consistent with the rest of the files. Same for other camelCase files

##########
File path: pulsar/schemaDef_test.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestSchemaDef(t *testing.T) {
+	errSchemaDef := "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," +

Review comment:
       Would ` marks make these easier to maintain? We would then not have to scape the " characters

##########
File path: pulsar/schema.go
##########
@@ -0,0 +1,502 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"reflect"
+	"unsafe"
+
+	log "github.com/sirupsen/logrus"
+
+	"github.com/gogo/protobuf/proto"
+	"github.com/linkedin/goavro"
+)
+
+type SchemaType int
+
+const (
+	NONE        SchemaType = iota //No schema defined
+	STRING                        //Simple String encoding with UTF-8
+	JSON                          //JSON object encoding and validation
+	PROTOBUF                      //Protobuf message encoding and decoding
+	AVRO                          //Serialize and deserialize via Avro
+	BOOLEAN                       //
+	INT8                          //A 8-byte integer.
+	INT16                         //A 16-byte integer.
+	INT32                         //A 32-byte integer.
+	INT64                         //A 64-byte integer.
+	FLOAT                         //A float number.
+	DOUBLE                        //A double number
+	_                             //
+	_                             //
+	_                             //
+	KeyValue                      //A Schema that contains Key Schema and Value Schema.
+	BYTES       = -1              //A bytes array.
+	AUTO        = -2              //
+	AutoConsume = -3              //Auto Consume Type.
+	AutoPublish = -4              // Auto Publish Type.
+)
+
+// Encapsulates data around the schema definition
+type SchemaInfo struct {
+	Name       string
+	Schema     string
+	Type       SchemaType
+	Properties map[string]string
+}
+
+type Schema interface {
+	Encode(v interface{}) ([]byte, error)
+	Decode(data []byte, v interface{}) error
+	Validate(message []byte) error
+	GetSchemaInfo() *SchemaInfo
+}
+
+type AvroCodec struct {
+	Codec *goavro.Codec
+}
+
+func NewSchemaDefinition(schema *goavro.Codec) *AvroCodec {
+	schemaDef := &AvroCodec{
+		Codec: schema,
+	}
+	return schemaDef
+}
+
+// initAvroCodec returns a Codec used to translate between a byte slice of either
+// binary or textual Avro data and native Go data.
+func initAvroCodec(codec string) (*goavro.Codec, error) {
+	return goavro.NewCodec(codec)
+}
+
+type JSONSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewJSONSchema(jsonAvroSchemaDef string, properties map[string]string) *JSONSchema {
+	js := new(JSONSchema)
+	avroCodec, err := initAvroCodec(jsonAvroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	js.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	js.SchemaInfo.Type = JSON
+	js.SchemaInfo.Properties = properties
+	js.SchemaInfo.Name = "Json"
+	return js
+}
+
+func (js *JSONSchema) Encode(data interface{}) ([]byte, error) {
+	return json.Marshal(data)
+}
+
+func (js *JSONSchema) Decode(data []byte, v interface{}) error {
+	return json.Unmarshal(data, v)
+}
+
+func (js *JSONSchema) Validate(message []byte) error {
+	return js.Decode(message, nil)
+}
+
+func (js *JSONSchema) GetSchemaInfo() *SchemaInfo {
+	return &js.SchemaInfo
+}
+
+type ProtoSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewProtoSchema(protoAvroSchemaDef string, properties map[string]string) *ProtoSchema {
+	ps := new(ProtoSchema)
+	avroCodec, err := initAvroCodec(protoAvroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	ps.AvroCodec.Codec = schemaDef.Codec
+	ps.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	ps.SchemaInfo.Type = PROTOBUF
+	ps.SchemaInfo.Properties = properties
+	ps.SchemaInfo.Name = "Proto"
+	return ps
+}
+
+func (ps *ProtoSchema) Encode(data interface{}) ([]byte, error) {
+	return proto.Marshal(data.(proto.Message))
+}
+
+func (ps *ProtoSchema) Decode(data []byte, v interface{}) error {
+	return proto.Unmarshal(data, v.(proto.Message))
+}
+
+func (ps *ProtoSchema) Validate(message []byte) error {
+	return ps.Decode(message, nil)
+}
+
+func (ps *ProtoSchema) GetSchemaInfo() *SchemaInfo {
+	return &ps.SchemaInfo
+}
+
+type AvroSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewAvroSchema(avroSchemaDef string, properties map[string]string) *AvroSchema {
+	as := new(AvroSchema)
+	avroCodec, err := initAvroCodec(avroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	as.AvroCodec.Codec = schemaDef.Codec
+	as.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	as.SchemaInfo.Type = AVRO
+	as.SchemaInfo.Name = "Avro"
+	as.SchemaInfo.Properties = properties
+	return as
+}
+
+func (as *AvroSchema) Encode(data interface{}) ([]byte, error) {
+	textual, err := json.Marshal(data)
+	if err != nil {
+		log.Errorf("serialize data error:%s", err.Error())
+		return nil, err
+	}
+	native, _, err := as.Codec.NativeFromTextual(textual)
+	if err != nil {
+		log.Errorf("convert native Go form to binary Avro data error:%s", err.Error())
+		return nil, err
+	}
+	return as.Codec.BinaryFromNative(nil, native)
+}
+
+func (as *AvroSchema) Decode(data []byte, v interface{}) error {
+	native, _, err := as.Codec.NativeFromBinary(data)
+	if err != nil {
+		log.Errorf("convert binary Avro data back to native Go form error:%s", err.Error())
+		return err
+	}
+	textual, err := as.Codec.TextualFromNative(nil, native)
+	if err != nil {
+		log.Errorf("convert native Go form to textual Avro data error:%s", err.Error())
+		return err
+	}
+	err = json.Unmarshal(textual, v)
+	if err != nil {
+		log.Errorf("unSerialize textual error:%s", err.Error())
+		return err
+	}
+	return nil
+}
+
+func (as *AvroSchema) Validate(message []byte) error {
+	return as.Decode(message, nil)
+}
+
+func (as *AvroSchema) GetSchemaInfo() *SchemaInfo {
+	return &as.SchemaInfo
+}
+
+type StringSchema struct {
+	SchemaInfo
+}
+
+func NewStringSchema(properties map[string]string) *StringSchema {
+	strSchema := new(StringSchema)
+	strSchema.SchemaInfo.Properties = properties
+	strSchema.SchemaInfo.Name = "String"
+	strSchema.SchemaInfo.Type = STRING
+	strSchema.SchemaInfo.Schema = ""
+	return strSchema
+}
+
+func (ss *StringSchema) Encode(v interface{}) ([]byte, error) {
+	return []byte(v.(string)), nil
+}
+
+func (ss *StringSchema) Decode(data []byte, v interface{}) error {
+	bh := (*reflect.SliceHeader)(unsafe.Pointer(&data))
+	sh := reflect.StringHeader{
+		Data: bh.Data,
+		Len:  bh.Len,
+	}
+	shPtr := (*string)(unsafe.Pointer(&sh))
+	reflect.ValueOf(v).Elem().Set(reflect.ValueOf(shPtr))
+	return nil
+}
+
+func (ss *StringSchema) Validate(message []byte) error {
+	return ss.Decode(message, nil)
+}
+
+func (ss *StringSchema) GetSchemaInfo() *SchemaInfo {
+	return &ss.SchemaInfo
+}
+
+type BytesSchema struct {
+	SchemaInfo
+}
+
+func NewBytesSchema(properties map[string]string) *BytesSchema {
+	bytesSchema := new(BytesSchema)
+	bytesSchema.SchemaInfo.Properties = properties
+	bytesSchema.SchemaInfo.Name = "Bytes"
+	bytesSchema.SchemaInfo.Type = BYTES
+	bytesSchema.SchemaInfo.Schema = ""
+	return bytesSchema
+}
+
+func (bs *BytesSchema) Encode(data interface{}) ([]byte, error) {
+	return data.([]byte), nil
+}
+
+func (bs *BytesSchema) Decode(data []byte, v interface{}) error {
+	reflect.ValueOf(v).Elem().Set(reflect.ValueOf(data))
+	return nil
+}
+
+func (bs *BytesSchema) Validate(message []byte) error {
+	return bs.Decode(message, nil)
+}
+
+func (bs *BytesSchema) GetSchemaInfo() *SchemaInfo {
+	return &bs.SchemaInfo
+}
+
+type Int8Schema struct {
+	SchemaInfo
+}
+
+func NewInt8Schema(properties map[string]string) *Int8Schema {
+	int8Schema := new(Int8Schema)
+	int8Schema.SchemaInfo.Properties = properties
+	int8Schema.SchemaInfo.Schema = ""
+	int8Schema.SchemaInfo.Type = INT8
+	int8Schema.SchemaInfo.Name = "INT8"
+	return int8Schema
+}
+
+func (is8 *Int8Schema) Encode(value interface{}) ([]byte, error) {
+	var buf bytes.Buffer
+	err := WriteElements(&buf, value.(int8))
+	return buf.Bytes(), err
+}
+
+func (is8 *Int8Schema) Decode(data []byte, v interface{}) error {
+	buf := bytes.NewReader(data)
+	return ReadElements(buf, v)
+}
+
+func (is8 *Int8Schema) Validate(message []byte) error {
+	if len(message) != 1 {
+		return errors.New("size of data received by Int8Schema is not 1")
+	}
+	return nil
+}
+
+func (is8 *Int8Schema) GetSchemaInfo() *SchemaInfo {
+	return &is8.SchemaInfo
+}
+
+type Int16Schema struct {
+	SchemaInfo
+}
+
+func NewInt16Schema(properties map[string]string) *Int16Schema {
+	int16Schema := new(Int16Schema)
+	int16Schema.SchemaInfo.Properties = properties
+	int16Schema.SchemaInfo.Name = "INT16"
+	int16Schema.SchemaInfo.Type = INT16
+	int16Schema.SchemaInfo.Schema = ""

Review comment:
       Why is the schema "" for all primitives?

##########
File path: pulsar/schema.go
##########
@@ -0,0 +1,502 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"reflect"
+	"unsafe"
+
+	log "github.com/sirupsen/logrus"
+
+	"github.com/gogo/protobuf/proto"
+	"github.com/linkedin/goavro"
+)
+
+type SchemaType int

Review comment:
       Should we add a String() method on the enum then for the names we can call JSON.String(), etc... and it's useful for logging and or debugging

##########
File path: pulsar/primitiveSerDe.go
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"encoding/binary"
+	"fmt"
+	"io"
+	"math"
+)
+
+const (
+	IoMaxSize     = 1024
+	maxBorrowSize = 10
+)
+
+var (
+	littleEndian = binary.LittleEndian
+)
+
+type BinaryFreeList chan []byte

Review comment:
       What's the purpose of this?

##########
File path: pulsar/message.go
##########
@@ -27,6 +27,9 @@ type ProducerMessage struct {
 	// Payload for the message
 	Payload []byte
 
+	//Value and payload is mutually exclusive, `Value interface{}` for schema message.
+	Value interface{}

Review comment:
       Should this be package private?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy edited a comment on pull request #368: Add schema logic in producer and consumer

Posted by GitBox <gi...@apache.org>.
wolfstudy edited a comment on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-714250012


   > @wolfstudy is there any specific reason why we are using v1.0.5 of goavro ? According to README `"github.com/linkedin/goavro/v2"` has better support for namespaces and performance improvements. Can we please switch to v2 ?
   
   Cool, I think we can update the `goavro` to `v2`, this looks good to me


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#issuecomment-706813612


   @merlimat @cckellogg PTAL thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503661401



##########
File path: pulsar/schema.go
##########
@@ -0,0 +1,502 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"reflect"
+	"unsafe"
+
+	log "github.com/sirupsen/logrus"
+
+	"github.com/gogo/protobuf/proto"
+	"github.com/linkedin/goavro"
+)
+
+type SchemaType int
+
+const (
+	NONE        SchemaType = iota //No schema defined
+	STRING                        //Simple String encoding with UTF-8
+	JSON                          //JSON object encoding and validation
+	PROTOBUF                      //Protobuf message encoding and decoding
+	AVRO                          //Serialize and deserialize via Avro
+	BOOLEAN                       //
+	INT8                          //A 8-byte integer.
+	INT16                         //A 16-byte integer.
+	INT32                         //A 32-byte integer.
+	INT64                         //A 64-byte integer.
+	FLOAT                         //A float number.
+	DOUBLE                        //A double number
+	_                             //
+	_                             //
+	_                             //
+	KeyValue                      //A Schema that contains Key Schema and Value Schema.
+	BYTES       = -1              //A bytes array.
+	AUTO        = -2              //
+	AutoConsume = -3              //Auto Consume Type.
+	AutoPublish = -4              // Auto Publish Type.
+)
+
+// Encapsulates data around the schema definition
+type SchemaInfo struct {
+	Name       string
+	Schema     string
+	Type       SchemaType
+	Properties map[string]string
+}
+
+type Schema interface {
+	Encode(v interface{}) ([]byte, error)
+	Decode(data []byte, v interface{}) error
+	Validate(message []byte) error
+	GetSchemaInfo() *SchemaInfo
+}
+
+type AvroCodec struct {
+	Codec *goavro.Codec
+}
+
+func NewSchemaDefinition(schema *goavro.Codec) *AvroCodec {
+	schemaDef := &AvroCodec{
+		Codec: schema,
+	}
+	return schemaDef
+}
+
+// initAvroCodec returns a Codec used to translate between a byte slice of either
+// binary or textual Avro data and native Go data.
+func initAvroCodec(codec string) (*goavro.Codec, error) {
+	return goavro.NewCodec(codec)
+}
+
+type JSONSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewJSONSchema(jsonAvroSchemaDef string, properties map[string]string) *JSONSchema {
+	js := new(JSONSchema)
+	avroCodec, err := initAvroCodec(jsonAvroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	js.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	js.SchemaInfo.Type = JSON
+	js.SchemaInfo.Properties = properties
+	js.SchemaInfo.Name = "Json"
+	return js
+}
+
+func (js *JSONSchema) Encode(data interface{}) ([]byte, error) {
+	return json.Marshal(data)
+}
+
+func (js *JSONSchema) Decode(data []byte, v interface{}) error {
+	return json.Unmarshal(data, v)
+}
+
+func (js *JSONSchema) Validate(message []byte) error {
+	return js.Decode(message, nil)
+}
+
+func (js *JSONSchema) GetSchemaInfo() *SchemaInfo {
+	return &js.SchemaInfo
+}
+
+type ProtoSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewProtoSchema(protoAvroSchemaDef string, properties map[string]string) *ProtoSchema {
+	ps := new(ProtoSchema)
+	avroCodec, err := initAvroCodec(protoAvroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	ps.AvroCodec.Codec = schemaDef.Codec
+	ps.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	ps.SchemaInfo.Type = PROTOBUF
+	ps.SchemaInfo.Properties = properties
+	ps.SchemaInfo.Name = "Proto"
+	return ps
+}
+
+func (ps *ProtoSchema) Encode(data interface{}) ([]byte, error) {
+	return proto.Marshal(data.(proto.Message))
+}
+
+func (ps *ProtoSchema) Decode(data []byte, v interface{}) error {
+	return proto.Unmarshal(data, v.(proto.Message))
+}
+
+func (ps *ProtoSchema) Validate(message []byte) error {
+	return ps.Decode(message, nil)
+}
+
+func (ps *ProtoSchema) GetSchemaInfo() *SchemaInfo {
+	return &ps.SchemaInfo
+}
+
+type AvroSchema struct {
+	AvroCodec
+	SchemaInfo
+}
+
+func NewAvroSchema(avroSchemaDef string, properties map[string]string) *AvroSchema {
+	as := new(AvroSchema)
+	avroCodec, err := initAvroCodec(avroSchemaDef)
+	if err != nil {
+		log.Fatalf("init codec error:%v", err)
+	}
+	schemaDef := NewSchemaDefinition(avroCodec)
+	as.AvroCodec.Codec = schemaDef.Codec
+	as.SchemaInfo.Schema = schemaDef.Codec.Schema()
+	as.SchemaInfo.Type = AVRO
+	as.SchemaInfo.Name = "Avro"
+	as.SchemaInfo.Properties = properties
+	return as
+}
+
+func (as *AvroSchema) Encode(data interface{}) ([]byte, error) {
+	textual, err := json.Marshal(data)
+	if err != nil {
+		log.Errorf("serialize data error:%s", err.Error())
+		return nil, err
+	}
+	native, _, err := as.Codec.NativeFromTextual(textual)
+	if err != nil {
+		log.Errorf("convert native Go form to binary Avro data error:%s", err.Error())
+		return nil, err
+	}
+	return as.Codec.BinaryFromNative(nil, native)
+}
+
+func (as *AvroSchema) Decode(data []byte, v interface{}) error {
+	native, _, err := as.Codec.NativeFromBinary(data)
+	if err != nil {
+		log.Errorf("convert binary Avro data back to native Go form error:%s", err.Error())
+		return err
+	}
+	textual, err := as.Codec.TextualFromNative(nil, native)
+	if err != nil {
+		log.Errorf("convert native Go form to textual Avro data error:%s", err.Error())
+		return err
+	}
+	err = json.Unmarshal(textual, v)
+	if err != nil {
+		log.Errorf("unSerialize textual error:%s", err.Error())
+		return err
+	}
+	return nil
+}
+
+func (as *AvroSchema) Validate(message []byte) error {
+	return as.Decode(message, nil)
+}
+
+func (as *AvroSchema) GetSchemaInfo() *SchemaInfo {
+	return &as.SchemaInfo
+}
+
+type StringSchema struct {
+	SchemaInfo
+}
+
+func NewStringSchema(properties map[string]string) *StringSchema {
+	strSchema := new(StringSchema)
+	strSchema.SchemaInfo.Properties = properties
+	strSchema.SchemaInfo.Name = "String"
+	strSchema.SchemaInfo.Type = STRING
+	strSchema.SchemaInfo.Schema = ""
+	return strSchema
+}
+
+func (ss *StringSchema) Encode(v interface{}) ([]byte, error) {
+	return []byte(v.(string)), nil
+}
+
+func (ss *StringSchema) Decode(data []byte, v interface{}) error {
+	bh := (*reflect.SliceHeader)(unsafe.Pointer(&data))
+	sh := reflect.StringHeader{
+		Data: bh.Data,
+		Len:  bh.Len,
+	}
+	shPtr := (*string)(unsafe.Pointer(&sh))
+	reflect.ValueOf(v).Elem().Set(reflect.ValueOf(shPtr))
+	return nil
+}
+
+func (ss *StringSchema) Validate(message []byte) error {
+	return ss.Decode(message, nil)
+}
+
+func (ss *StringSchema) GetSchemaInfo() *SchemaInfo {
+	return &ss.SchemaInfo
+}
+
+type BytesSchema struct {
+	SchemaInfo
+}
+
+func NewBytesSchema(properties map[string]string) *BytesSchema {
+	bytesSchema := new(BytesSchema)
+	bytesSchema.SchemaInfo.Properties = properties
+	bytesSchema.SchemaInfo.Name = "Bytes"
+	bytesSchema.SchemaInfo.Type = BYTES
+	bytesSchema.SchemaInfo.Schema = ""
+	return bytesSchema
+}
+
+func (bs *BytesSchema) Encode(data interface{}) ([]byte, error) {
+	return data.([]byte), nil
+}
+
+func (bs *BytesSchema) Decode(data []byte, v interface{}) error {
+	reflect.ValueOf(v).Elem().Set(reflect.ValueOf(data))
+	return nil
+}
+
+func (bs *BytesSchema) Validate(message []byte) error {
+	return bs.Decode(message, nil)
+}
+
+func (bs *BytesSchema) GetSchemaInfo() *SchemaInfo {
+	return &bs.SchemaInfo
+}
+
+type Int8Schema struct {
+	SchemaInfo
+}
+
+func NewInt8Schema(properties map[string]string) *Int8Schema {
+	int8Schema := new(Int8Schema)
+	int8Schema.SchemaInfo.Properties = properties
+	int8Schema.SchemaInfo.Schema = ""
+	int8Schema.SchemaInfo.Type = INT8
+	int8Schema.SchemaInfo.Name = "INT8"
+	return int8Schema
+}
+
+func (is8 *Int8Schema) Encode(value interface{}) ([]byte, error) {
+	var buf bytes.Buffer
+	err := WriteElements(&buf, value.(int8))
+	return buf.Bytes(), err
+}
+
+func (is8 *Int8Schema) Decode(data []byte, v interface{}) error {
+	buf := bytes.NewReader(data)
+	return ReadElements(buf, v)
+}
+
+func (is8 *Int8Schema) Validate(message []byte) error {
+	if len(message) != 1 {
+		return errors.New("size of data received by Int8Schema is not 1")
+	}
+	return nil
+}
+
+func (is8 *Int8Schema) GetSchemaInfo() *SchemaInfo {
+	return &is8.SchemaInfo
+}
+
+type Int16Schema struct {
+	SchemaInfo
+}
+
+func NewInt16Schema(properties map[string]string) *Int16Schema {
+	int16Schema := new(Int16Schema)
+	int16Schema.SchemaInfo.Properties = properties
+	int16Schema.SchemaInfo.Name = "INT16"
+	int16Schema.SchemaInfo.Type = INT16
+	int16Schema.SchemaInfo.Schema = ""

Review comment:
       For primitives types, the schema field should be `""`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [pulsar-client-go] wolfstudy commented on a change in pull request #368: Package basic tool functions for schema logic

Posted by GitBox <gi...@apache.org>.
wolfstudy commented on a change in pull request #368:
URL: https://github.com/apache/pulsar-client-go/pull/368#discussion_r503659298



##########
File path: pulsar/schemaDef_test.go
##########
@@ -0,0 +1,57 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package pulsar
+
+import (
+	"testing"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func TestSchemaDef(t *testing.T) {
+	errSchemaDef := "{\"type\":\"record\",\"name\":\"Example\",\"namespace\":\"test\"," +

Review comment:
       Good ideas will fix it




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org