You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@pulsar.apache.org by GitBox <gi...@apache.org> on 2020/12/29 09:13:10 UTC

[GitHub] [pulsar-client-node] tuteng opened a new pull request #138: Support end to end encryption for node client

tuteng opened a new pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138


   
   After this pr is https://github.com/apache/pulsar/pull/9074 merged, we can consider enabling end-to-end encryption for the node client.
   
   * Enabling end-to-end encryption
   * Add producer, consumer examples and test certificates


----------------------------------------------------------------
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559931207



##########
File path: src/ConsumerConfig.cc
##########
@@ -163,6 +172,23 @@ ConsumerConfig::ConsumerConfig(const Napi::Object &consumerConfig,
       pulsar_consumer_set_read_compacted(this->cConsumerConfig, 1);
     }
   }
+
+  if ((consumerConfig.Has(CFG_PUBLIC_KEY_PATH) && consumerConfig.Has(CFG_PRIVATE_KEY_PATH)) &&

Review comment:
       Is the public key required on the consumer side?




----------------------------------------------------------------
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-node] tuteng commented on pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#issuecomment-806461974


   The test failed due to a deb package issue, which will be fixed later


-- 
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559927182



##########
File path: examples/encryption-producer.js
##########
@@ -0,0 +1,52 @@
+/**
+ * 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.
+ */
+
+const Pulsar = require('pulsar-client');
+
+(async () => {
+  // Create a client
+  const client = new Pulsar.Client({
+    serviceUrl: 'pulsar://localhost:6650',
+    operationTimeoutSeconds: 30,
+  });
+
+  // Create a producer
+  const producer = await client.createProducer({
+    topic: 'persistent://public/default/my-topic',
+    sendTimeoutMs: 30000,
+    batchingEnabled: true,
+    publicKeyPath: "./certificate/public-key.client-rsa.pem",
+    privateKeyPath: "./certificate/private-key.client-rsa.pem",
+    encryptionKey: "encryption-key"
+  });
+
+  console.log(producer.ProducerConfig)

Review comment:
       `producer.ProducerConfig` is undefined.




----------------------------------------------------------------
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-node] massakam merged pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam merged pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138


   


-- 
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559931207



##########
File path: src/ConsumerConfig.cc
##########
@@ -163,6 +172,23 @@ ConsumerConfig::ConsumerConfig(const Napi::Object &consumerConfig,
       pulsar_consumer_set_read_compacted(this->cConsumerConfig, 1);
     }
   }
+
+  if ((consumerConfig.Has(CFG_PUBLIC_KEY_PATH) && consumerConfig.Has(CFG_PRIVATE_KEY_PATH)) &&

Review comment:
       Is a public key required on the consumer side?




----------------------------------------------------------------
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559980339



##########
File path: tests/end_to_end.test.js
##########
@@ -665,5 +669,87 @@ const Pulsar = require('../index.js');
       await consumer.close();
       await client.close();
     });
+    test('Basic produce and consume encryption', async () => {
+      const client = new Pulsar.Client({
+        serviceUrl: 'pulsar://localhost:6650',
+        operationTimeoutSeconds: 30,
+      });
+
+      const topic = 'persistent://public/default/encryption-produce-consume';
+      const producer = await client.createProducer({
+        topic,
+        sendTimeoutMs: 30000,
+        batchingEnabled: true,
+        publicKeyPath: './certificate/public-rsa.pem',
+        privateKeyPath: './certificate/private-rsa.pem',
+        encryptionKey: 'encryption-key',
+      });
+
+      const consumer = await client.subscribe({
+        topic,
+        subscription: 'sub1',
+        subscriptionType: 'Shared',
+        ackTimeoutMs: 10000,
+        publicKeyPath: './public-rsa.pem',
+        privateKeyPath: './private-rsa.pem',
+      });
+
+      const messages = [];
+      for (let i = 0; i < 10; i += 1) {
+        const msg = `my-message-${i}`;
+        producer.send({
+          data: Buffer.from(msg),
+        });
+        messages.push(msg);
+      }
+      await producer.flush();
+
+      const results = [];
+      for (let i = 0; i < 10; i += 1) {
+        const msg = await consumer.receive();
+        consumer.acknowledge(msg);
+        results.push(msg.getData().toString());
+      }
+      expect(lodash.difference(messages, results)).toEqual([]);
+      await producer.close();
+      await consumer.close();
+      await client.close();
+    });
+
+    test('Failed produce and consume encryption', async () => {

Review comment:
       Is this test case necessary? It doesn't seem to be a test related to encryption.




----------------------------------------------------------------
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-node] tuteng commented on pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#issuecomment-819199304


   @massakam PTAL


-- 
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559975275



##########
File path: src/ProducerConfig.cc
##########
@@ -153,6 +162,25 @@ ProducerConfig::ProducerConfig(const Napi::Object& producerConfig) : topic("") {
                                                  value.Utf8Value().c_str());
     }
   }
+
+  if ((producerConfig.Has(CFG_PUBLIC_KEY_PATH) && producerConfig.Has(CFG_PRIVATE_KEY_PATH)) &&
+      (producerConfig.Get(CFG_PUBLIC_KEY_PATH).IsString() &&
+       producerConfig.Get(CFG_PRIVATE_KEY_PATH).IsString())) {
+    std::string publicKeyPath = producerConfig.Get(CFG_PUBLIC_KEY_PATH).ToString().Utf8Value();
+    std::string privateKeyPath = producerConfig.Get(CFG_PRIVATE_KEY_PATH).ToString().Utf8Value();
+    pulsar_producer_configuration_set_default_crypto_key_reader(this->cProducerConfig, publicKeyPath.c_str(),
+                                                                privateKeyPath.c_str());
+    if (producerConfig.Has(CFG_ENCRYPTION_KEY) && producerConfig.Get(CFG_ENCRYPTION_KEY).IsString()) {
+      std::string encryptionKey = producerConfig.Get(CFG_ENCRYPTION_KEY).ToString().Utf8Value();
+      pulsar_producer_configuration_set_encryption_key(this->cProducerConfig, encryptionKey.c_str());
+    }
+    if (producerConfig.Has(CFG_CRYPTO_FAILURE_ACTION) &&
+        producerConfig.Get(CFG_CRYPTO_FAILURE_ACTION).IsString()) {
+      std::string cryptoFailureAction = producerConfig.Get(CFG_CRYPTO_FAILURE_ACTION).ToString().Utf8Value();
+      pulsar_producer_configuration_set_crypto_failure_action(
+          this->cProducerConfig, PRODUCER_CRYPTO_FAILURE_ACTION.at(cryptoFailureAction));

Review comment:
       We should check if `cryptoFailureAction` is contained in `PRODUCER_CRYPTO_FAILURE_ACTION`.




----------------------------------------------------------------
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559973025



##########
File path: src/ProducerConfig.cc
##########
@@ -153,6 +162,25 @@ ProducerConfig::ProducerConfig(const Napi::Object& producerConfig) : topic("") {
                                                  value.Utf8Value().c_str());
     }
   }
+
+  if ((producerConfig.Has(CFG_PUBLIC_KEY_PATH) && producerConfig.Has(CFG_PRIVATE_KEY_PATH)) &&

Review comment:
       On the producer side, a private key does not seem to be required.




----------------------------------------------------------------
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-node] tuteng commented on pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
tuteng commented on pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#issuecomment-806439444


   Now that 2.7.1 has been released, this feature is available, I fixed your comment, PTAL @massakam 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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559926904



##########
File path: examples/encryption-consumer.js
##########
@@ -0,0 +1,49 @@
+/**
+ * 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.
+ */
+
+const Pulsar = require('pulsar-client');
+
+(async () => {
+  // Create a client
+  const client = new Pulsar.Client({
+    serviceUrl: 'pulsar://localhost:6650',
+    operationTimeoutSeconds: 30
+  });
+
+  // Create a consumer
+  const consumer = await client.subscribe({
+    topic: 'persistent://public/default/my-topic',
+    subscription: 'sub1',
+    subscriptionType: 'Shared',
+    ackTimeoutMs: 10000,
+    publicKeyPath: "./certificate/public-key.client-rsa.pem",
+    privateKeyPath: "./certificate/private-key.client-rsa.pem"
+  });
+
+  console.log(consumer)

Review comment:
       It seems that no useful information is output.
   ```
   Consumer {}
   ```




----------------------------------------------------------------
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-node] massakam commented on pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#issuecomment-762102019


   The milestone for https://github.com/apache/pulsar/pull/9074 is 2.7.1 and has not yet been released. We cannot merge this PR until version 2.7.1 of the C/C++ client library is released.


----------------------------------------------------------------
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-node] massakam commented on a change in pull request #138: Support end to end encryption for node client

Posted by GitBox <gi...@apache.org>.
massakam commented on a change in pull request #138:
URL: https://github.com/apache/pulsar-client-node/pull/138#discussion_r559977594



##########
File path: tests/end_to_end.test.js
##########
@@ -665,5 +669,87 @@ const Pulsar = require('../index.js');
       await consumer.close();
       await client.close();
     });
+    test('Basic produce and consume encryption', async () => {
+      const client = new Pulsar.Client({
+        serviceUrl: 'pulsar://localhost:6650',
+        operationTimeoutSeconds: 30,
+      });
+
+      const topic = 'persistent://public/default/encryption-produce-consume';
+      const producer = await client.createProducer({
+        topic,
+        sendTimeoutMs: 30000,
+        batchingEnabled: true,
+        publicKeyPath: './certificate/public-rsa.pem',
+        privateKeyPath: './certificate/private-rsa.pem',
+        encryptionKey: 'encryption-key',
+      });
+
+      const consumer = await client.subscribe({
+        topic,
+        subscription: 'sub1',
+        subscriptionType: 'Shared',
+        ackTimeoutMs: 10000,
+        publicKeyPath: './public-rsa.pem',
+        privateKeyPath: './private-rsa.pem',

Review comment:
       Are these paths correct?




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