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 2021/06/29 01:31:42 UTC

[GitHub] [pulsar-client-go] cckellogg commented on a change in pull request #555: add base package for encryption

cckellogg commented on a change in pull request #555:
URL: https://github.com/apache/pulsar-client-go/pull/555#discussion_r660206561



##########
File path: pulsar/crypto/encryption_key_Info.go
##########
@@ -0,0 +1,64 @@
+// 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 crypto
+
+// EncryptionKeyInfo
+type EncryptionKeyInfo struct {
+	metadata map[string]string
+	value    []byte
+	key      string
+}
+
+// NewEncryptionKeyInfo
+func NewEncryptionKeyInfo(key string, value []byte, metadata map[string]string) *EncryptionKeyInfo {
+	return &EncryptionKeyInfo{
+		metadata: metadata,
+		key:      key,
+		value:    value,
+	}
+}
+
+// GetKey get key
+func (eci *EncryptionKeyInfo) GetKey() string {
+	return eci.key
+}
+
+// SetKey set key
+func (eci *EncryptionKeyInfo) SetKey(key string) {

Review comment:
       Why are set methods needed? Should we make this immutable and if changes are needed create a new struct.

##########
File path: pulsar/crypto/default_message_crypto.go
##########
@@ -0,0 +1,379 @@
+// 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 crypto
+
+import (
+	gocrypto "crypto"
+	"crypto/aes"
+	"crypto/cipher"
+	"crypto/md5"
+	"crypto/rand"
+	"crypto/rsa"
+	"crypto/sha1"
+	"crypto/x509"
+	"encoding/pem"
+	"errors"
+	"fmt"
+	"sync"
+
+	"github.com/apache/pulsar-client-go/pulsar/log"
+)
+
+// DefaultMessageCrypto implmentation of the interface MessageCryto
+type DefaultMessageCrypto struct {
+	// data key which is used to encrypt/decrypt messages
+	dataKey []byte
+
+	// LoadingCache used by the consumer to cache already decrypted key
+	loadingCache sync.Map // map[string][]byte
+
+	encryptedDataKeyMap sync.Map // map[string]EncryptionKeyInfo
+
+	logCtx string
+
+	logger log.Logger
+
+	cipherLock sync.Mutex
+
+	encryptLock sync.Mutex
+}
+
+const (
+	errGeneratingNonce          = "failed to generate new nonce"
+	errEmptyKeyNameOrKeyReader  = "keyname or keyreader is null"
+	errParseRSAPubKey           = "unable to parse RSA public key"
+	errParseRSAPriKey           = "unable to parse RSA private key"
+	errEncryptionKeyInfo        = "Failed to get EncryptionKeyInfo for key %v"
+	errEncryptedDataKeyNotFound = "%v Failed to find encrypted Data key for key %v"
+	errAESCipherCreation        = "failed to create AES cipher"
+	errGCMCreation              = "failed to create gcm"
+	errDataKeyDecryption        = "unable to decrypt data key"
+	errDecodePriKey             = "failed to decode private key"
+	errDecodePubKey             = "failed to decode public key"
+)
+
+// NewDefaultMessageCrypto get the instance of message crypto
+func NewDefaultMessageCrypto(logCtx string, keyGenNeeded bool, logger log.Logger) (*DefaultMessageCrypto, error) {
+
+	d := &DefaultMessageCrypto{
+		logCtx:              logCtx,
+		loadingCache:        sync.Map{},
+		encryptedDataKeyMap: sync.Map{},
+		logger:              logger,
+	}
+
+	if keyGenNeeded {
+		key, err := generateDataKey()
+		if err != nil {
+			return d, err
+		}
+		d.dataKey = key
+	}
+
+	return d, nil
+}
+
+// AddPublicKeyCipher encrypt data key using keyCrypto and cache
+func (d *DefaultMessageCrypto) AddPublicKeyCipher(keyNames []string, keyReader KeyReader) error {
+	key, err := generateDataKey()
+	if err != nil {
+		return err
+	}
+
+	d.dataKey = key
+	for _, keyName := range keyNames {
+		err := d.addPublicKeyCipher(keyName, keyReader)
+		if err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (d *DefaultMessageCrypto) addPublicKeyCipher(keyName string, keyReader KeyReader) error {
+	d.cipherLock.Lock()
+	defer d.cipherLock.Unlock()
+	if keyName == "" || keyReader == nil {
+		return fmt.Errorf(errEmptyKeyNameOrKeyReader)
+	}
+
+	// read the public key and its info using keyReader
+	keyInfo, err := keyReader.GetPublicKey(keyName, nil)
+	if err != nil {
+		return err
+	}
+
+	parsedKey, err := d.loadPublicKey(keyInfo.GetValue())
+	if err != nil {
+		return err
+	}
+
+	// try to cast to RSA key
+	rsaPubKey, ok := parsedKey.(*rsa.PublicKey)

Review comment:
       Why check for RSA here? And if that is only supported should we return an error like expected a RSA key or only RSA keys are supported?

##########
File path: pulsar/crypto/crypto_key_reader.go
##########
@@ -0,0 +1,28 @@
+// 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 crypto
+
+// KeyReader implement this interface to read and provide public & private keys
+// key pair can be RSA, ECDSA
+type KeyReader interface {
+	// GetPublicKey get public key that is be used by the producer to encrypt data key
+	GetPublicKey(keyName string, metadata map[string]string) (*EncryptionKeyInfo, error)
+
+	// GetPrivateKey get private key that is used by the consumer to decrypt data key
+	GetPrivateKey(keyName string, metadata map[string]string) (*EncryptionKeyInfo, error)

Review comment:
       Nitpick generally go code style does not use get for values. It would be PublicKey() and PrivateKey() instead. Same for other parts fo the code.

##########
File path: pulsar/crypto/pri_key_rsa.pem
##########
@@ -0,0 +1,39 @@
+-----BEGIN RSA PRIVATE KEY-----

Review comment:
       move add these test keys file to a pulsar/crypto/testdata directory
   
   https://golang.org/pkg/cmd/go/internal/test/
   ```
   The go tool will ignore a directory named "testdata", making it available
   to hold ancillary data needed by the tests.
   ```
   

##########
File path: pulsar/crypto/crypto_key_reader.go
##########
@@ -0,0 +1,28 @@
+// 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 crypto
+
+// KeyReader implement this interface to read and provide public & private keys
+// key pair can be RSA, ECDSA
+type KeyReader interface {
+	// GetPublicKey get public key that is be used by the producer to encrypt data key
+	GetPublicKey(keyName string, metadata map[string]string) (*EncryptionKeyInfo, error)

Review comment:
       Where does the key and metadata come from? the message properties?

##########
File path: pulsar/crypto/encryption_key_Info.go
##########
@@ -0,0 +1,64 @@
+// 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 crypto
+
+// EncryptionKeyInfo
+type EncryptionKeyInfo struct {
+	metadata map[string]string
+	value    []byte
+	key      string
+}
+
+// NewEncryptionKeyInfo
+func NewEncryptionKeyInfo(key string, value []byte, metadata map[string]string) *EncryptionKeyInfo {
+	return &EncryptionKeyInfo{
+		metadata: metadata,
+		key:      key,
+		value:    value,
+	}
+}
+
+// GetKey get key
+func (eci *EncryptionKeyInfo) GetKey() string {

Review comment:
       When i think of key i think of the actual bytes of the key what about just these functions
   ```
   Name() string
   Key() []byte
   Meta() map[string]string
   
   ```
   

##########
File path: pulsar/crypto/message_metadata.go
##########
@@ -0,0 +1,122 @@
+// 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 crypto
+
+import (
+	pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
+)
+
+// MessageMetadataSupplier wrapper implementation around message metadata
+type MessageMetadataSupplier interface {
+	// GetEncryptionKeys read all the encryption keys from the MessageMetadata
+	GetEncryptionKeys() []EncryptionKeyInfo
+
+	// UpsertEncryptionkey add new or update existing EncryptionKeys in to the MessageMetadata
+	UpsertEncryptionkey(EncryptionKeyInfo)
+
+	// GetEncryptionParam read the ecryption parameter from the MessageMetadata
+	GetEncryptionParam() []byte
+
+	// SetEncryptionParam set encryption parameter in to the MessageMetadata
+	SetEncryptionParam([]byte)

Review comment:
       Are setters need in this interface?

##########
File path: pulsar/crypto/crypto_failure_action.go
##########
@@ -0,0 +1,46 @@
+// 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 crypto
+
+type ProducerCryptoFailureAction int
+type ConsumerCryptoFailureAction int
+
+const (

Review comment:
       Since there aren't really enums in go to namespace things maybe we should use long names to make it easier to reason about in the code?
   
   ProducerCryptoFailureActionFail or ProducerFailureActionSend
   ProducerCryptoFailureActionSend or ProducerFailureActionFail
   
   Thoughts?
   

##########
File path: pulsar/crypto/default_message_crypto.go
##########
@@ -0,0 +1,379 @@
+// 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 crypto
+
+import (
+	gocrypto "crypto"
+	"crypto/aes"
+	"crypto/cipher"
+	"crypto/md5"
+	"crypto/rand"
+	"crypto/rsa"
+	"crypto/sha1"
+	"crypto/x509"
+	"encoding/pem"
+	"errors"
+	"fmt"
+	"sync"
+
+	"github.com/apache/pulsar-client-go/pulsar/log"
+)
+
+// DefaultMessageCrypto implmentation of the interface MessageCryto
+type DefaultMessageCrypto struct {
+	// data key which is used to encrypt/decrypt messages
+	dataKey []byte
+
+	// LoadingCache used by the consumer to cache already decrypted key
+	loadingCache sync.Map // map[string][]byte
+
+	encryptedDataKeyMap sync.Map // map[string]EncryptionKeyInfo
+
+	logCtx string
+
+	logger log.Logger
+
+	cipherLock sync.Mutex
+
+	encryptLock sync.Mutex
+}
+
+const (
+	errGeneratingNonce          = "failed to generate new nonce"

Review comment:
       create error vars for strings that have no formatting and no need to create formatting constants if they are only used once. It's easier to get context when looking at the code if the formatting is done where the log/error is happening.
   
   ```
   var (
   	
   	errEmptyKeyNameOrKeyReader = fmt.Errorf("keyname or keyreader is not provided"),
           ...
   )
   ```

##########
File path: pulsar/crypto/message_metadata.go
##########
@@ -0,0 +1,122 @@
+// 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 crypto
+
+import (
+	pb "github.com/apache/pulsar-client-go/pulsar/internal/pulsar_proto"
+)
+
+// MessageMetadataSupplier wrapper implementation around message metadata
+type MessageMetadataSupplier interface {

Review comment:
       What is the goal of this class and am wondering if can come up with more meaningful name?




-- 
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.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

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