You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/09 14:03:59 UTC

incubator-ignite git commit: #ignite-961: add authentication for node js.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-961 488d2f648 -> 9c1039b04


#ignite-961: add authentication for node js.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/9c1039b0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9c1039b0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9c1039b0

Branch: refs/heads/ignite-961
Commit: 9c1039b04bc6f6e15b14f9b7be9e2cf96cd6dda8
Parents: 488d2f6
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Jun 9 15:03:51 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Jun 9 15:03:51 2015 +0300

----------------------------------------------------------------------
 modules/nodejs/src/main/js/ignition.js          | 14 ++--
 modules/nodejs/src/main/js/server.js            | 58 ++++++++++++++--
 .../ignite/internal/NodeJsCacheApiSelfTest.java |  2 -
 .../internal/NodeJsSecretKeySelfTest.java       | 72 ++++++++++++++++++++
 .../testsuites/IgniteNodeJsTestSuite.java       |  1 +
 modules/nodejs/src/test/js/test-ignition.js     |  8 +--
 modules/nodejs/src/test/js/test-key.js          | 62 +++++++++++++++++
 modules/nodejs/src/test/js/test-utils.js        | 14 +++-
 8 files changed, 210 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/main/js/ignition.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/ignition.js b/modules/nodejs/src/main/js/ignition.js
index fb98562..d641d26 100644
--- a/modules/nodejs/src/main/js/ignition.js
+++ b/modules/nodejs/src/main/js/ignition.js
@@ -35,9 +35,10 @@ function Ignition() {
  * Open connection with server node
  *
  * @param {string[]} address List of nodes hosts with ports
+ * @param {string} secretKey Secret key.
  * @param {Ignition~onStart} callback Called on finish
  */
-Ignition.start = function(address, callback) {
+Ignition.start = function(address, secretKey, callback) {
   var Server = require("./server").Server;
   var Ignite = require("./ignite").Ignite
 
@@ -66,14 +67,14 @@ Ignition.start = function(address, callback) {
     }
 
     for (var i = start; i <= end; i++) {
-      checkServer(params[0], i);
+      checkServer(params[0], i, secretKey);
     }
   }
 
-  function checkServer(host, port) {
+  function checkServer(host, port, secretKey) {
     numConn++;
 
-    var server = new Server(host, port);
+    var server = new Server(host, port, secretKey);
 
     server.checkConnection(onConnect.bind(null, server));
   }
@@ -88,6 +89,7 @@ Ignition.start = function(address, callback) {
     if (!callback) return;
 
     numConn--;
+
     if (!error) {
       callback.call(null, null, new Ignite(server));
 
@@ -96,10 +98,8 @@ Ignition.start = function(address, callback) {
       return;
     }
 
-    console.log("onConnect:" + error);
-
     if (!numConn) {
-      callback.call(null, "Cannot connect to servers.", null);
+      callback.call(null, "Cannot connect to servers. " + error, null);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/main/js/server.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/main/js/server.js b/modules/nodejs/src/main/js/server.js
index 224db8b..082627d 100644
--- a/modules/nodejs/src/main/js/server.js
+++ b/modules/nodejs/src/main/js/server.js
@@ -22,10 +22,12 @@
  * @this {Server}
  * @param {string} host Host address
  * @param {number} port Port
+ * @param {string} secretKey Secret key for connection
  */
-function Server(host, port) {
+function Server(host, port, secretKey) {
   this._host = host;
   this._port = port;
+  this._secretKey = secretKey;
 }
 
 /**
@@ -68,7 +70,8 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
   var options = {
     host: this._host,
     port: this._port,
-    path: "/ignite?" + requestQry
+    path: "/ignite?" + requestQry,
+    headers: this._signature()
   };
 
   function streamCallback(response) {
@@ -79,20 +82,32 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
     });
 
     response.on('end', function () {
-      var response;
+      if (response.statusCode !== 200) {
+        if (response.statusCode === 401) {
+          callback.call(null, "Authentication failed. Status code 401.");
+        }
+        else {
+          callback.call(null, "Request failed. Status code " + statusCode);
+        }
+
+        return;
+      }
+
+      var igniteResponse;
+
       try {
-        response = JSON.parse(fullResponseString);
+        igniteResponse = JSON.parse(fullResponseString);
       }
       catch (e) {
         callback.call(null, e, null);
         return;
       }
 
-      if (response.successStatus) {
-        callback.call(null, response.error, null)
+      if (igniteResponse.successStatus) {
+        callback.call(null, igniteResponse.error, null)
       }
       else {
-        callback.call(null, null, response.response);
+        callback.call(null, null, igniteResponse.response);
       }
     });
   }
@@ -102,6 +117,7 @@ Server.prototype.runCommand = function(cmdName, params, callback) {
   request.setTimeout(5000, callback.bind(null, "Request timeout: >5 sec"));
 
   request.on('error', callback);
+
   request.end();
 }
 
@@ -126,4 +142,32 @@ Server.pair = function(key, value) {
   return {key: key, value: value}
 }
 
+/**
+ * Get signature for connection.
+ *
+ * @this {Server}
+ * @returns Signature
+ */
+Server.prototype._signature = function() {
+  if (!this._secretKey) {
+    return "";
+  }
+
+  var loadTimeInMS = Date.now();
+
+  var baseKey = '' + loadTimeInMS + ":" + this._secretKey;
+
+  var crypto = require('crypto')
+
+  var shasum = crypto.createHash('sha1');
+
+  shasum.update(baseKey, 'binary');
+
+  var hash = shasum.digest('base64');
+
+  var key = loadTimeInMS + ":" + hash;
+
+  return {"X-Signature" : key};
+}
+
 exports.Server = Server;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
index 909282f..6c02ede 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsCacheApiSelfTest.java
@@ -39,8 +39,6 @@ public class NodeJsCacheApiSelfTest extends NodeJsAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         grid(0).cache(NodeJsAbstractTest.CACHE_NAME).removeAll();
-
-        assertNull(grid(0).cache(NodeJsAbstractTest.CACHE_NAME).get("key"));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSecretKeySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSecretKeySelfTest.java b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSecretKeySelfTest.java
new file mode 100644
index 0000000..ab7b961
--- /dev/null
+++ b/modules/nodejs/src/test/java/org/apache/ignite/internal/NodeJsSecretKeySelfTest.java
@@ -0,0 +1,72 @@
+/*
+ * 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 org.apache.ignite.internal;
+
+import org.apache.ignite.configuration.*;
+
+/**
+ * Test secret key.
+ */
+public class NodeJsSecretKeySelfTest extends NodeJsAbstractTest {
+    /**
+     * Constructor.
+     */
+    public NodeJsSecretKeySelfTest() {
+        super("test-key.js");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.getConnectorConfiguration().setSecretKey("secret-key");
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartWithoutKey() throws Exception {
+        runJsScript("testStartWithoutKey");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartWithKey() throws Exception {
+        runJsScript("testStartWithKey");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartWithIncorrectKey() throws Exception {
+        runJsScript("testStartWithIncorrectKey");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java b/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
index 2f853f6..9d5daee 100644
--- a/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
+++ b/modules/nodejs/src/test/java/org/apache/ignite/testsuites/IgniteNodeJsTestSuite.java
@@ -33,6 +33,7 @@ public class IgniteNodeJsTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(NodeJsIgnitionSelfTest.class));
         suite.addTest(new TestSuite(NodeJsCacheApiSelfTest.class));
+        suite.addTest(new TestSuite(NodeJsSecretKeySelfTest.class));
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/js/test-ignition.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-ignition.js b/modules/nodejs/src/test/js/test-ignition.js
index 9b7a931..c43e856 100644
--- a/modules/nodejs/src/test/js/test-ignition.js
+++ b/modules/nodejs/src/test/js/test-ignition.js
@@ -20,7 +20,7 @@ var Apache = require(TestUtils.scriptPath());
 var Ignition = Apache.Ignition;
 
 testIgnitionFail = function ()  {
-    Ignition.start(['127.0.0.3:9091', '127.0.0.1:9092'], onConnect);
+    Ignition.start(['127.0.0.3:9091', '127.0.0.1:9092'], null, onConnect);
 
     function onConnect(error, server) {
         if (error) {
@@ -37,7 +37,7 @@ testIgnitionFail = function ()  {
 }
 
 ignitionStartSuccess = function() {
-    Ignition.start(['127.0.0.0:9095', '127.0.0.1:9095'], onConnect);
+    Ignition.start(['127.0.0.0:9095', '127.0.0.1:9095'], null, onConnect);
 
     function onConnect(error, server) {
         if (error) {
@@ -51,7 +51,7 @@ ignitionStartSuccess = function() {
 }
 
 ignitionStartSuccessWithSeveralPorts = function() {
-    Ignition.start(['127.0.0.1:9090..9100'], onConnect);
+    Ignition.start(['127.0.0.1:9090..9100'], null, onConnect);
 
     function onConnect(error, ignite) {
         if (error) {
@@ -73,7 +73,7 @@ ignitionStartSuccessWithSeveralPorts = function() {
 }
 
 ignitionNotStartWithSeveralPorts = function() {
-    Ignition.start(['127.0.0.1:9090...9100'], onConnect);
+    Ignition.start(['127.0.0.1:9090...9100'], null, onConnect);
 
     function onConnect(error, ignite) {
         if (error) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/js/test-key.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-key.js b/modules/nodejs/src/test/js/test-key.js
new file mode 100644
index 0000000..b39aa51
--- /dev/null
+++ b/modules/nodejs/src/test/js/test-key.js
@@ -0,0 +1,62 @@
+/*
+ * 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.
+ */
+
+var TestUtils = require("./test-utils").TestUtils;
+
+var Apache = require(TestUtils.scriptPath());
+var Cache = Apache.Cache;
+var Server = Apache.Server;
+
+var assert = require("assert");
+
+testStartWithoutKey = function() {
+  TestUtils.startIgniteNode(onIncorrectStart);
+}
+
+testStartWithKey = function() {
+  TestUtils.startIgniteNodeWithKey("secret-key", onStart);
+}
+
+testStartWithIncorrectKey = function() {
+  TestUtils.startIgniteNodeWithKey("secret-key1", onIncorrectStart);
+}
+
+function onIncorrectStart(error, ignite) {
+  assert(error != null, "Do not get authentication error");
+
+  assert(error.indexOf("Authentication failed. Status code 401.") !== -1, "Incorrect error message: " + error);
+
+  TestUtils.testDone();
+}
+
+function onStart(error, ignite) {
+  assert(error === null, "Get error: " + error);
+
+  assert(ignite !== null, "Cannot connect. Get null ignite.");
+
+  var cache = ignite.cache("mycache");
+
+  assert(cache !== null, "Cache is null.")
+
+  cache.put("key", "6", onPut);
+}
+
+function onPut(error) {
+    assert(error === null, "Error on put:" + error);
+
+    TestUtils.testDone();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9c1039b0/modules/nodejs/src/test/js/test-utils.js
----------------------------------------------------------------------
diff --git a/modules/nodejs/src/test/js/test-utils.js b/modules/nodejs/src/test/js/test-utils.js
index 0120690..135da8c 100644
--- a/modules/nodejs/src/test/js/test-utils.js
+++ b/modules/nodejs/src/test/js/test-utils.js
@@ -131,7 +131,19 @@ TestUtils.testDone = function() {
 TestUtils.startIgniteNode = function(callback) {
   var Apache = require(TestUtils.scriptPath());
   var Ignition = Apache.Ignition;
-  Ignition.start(['127.0.0.1:9095'], callback);
+  Ignition.start(['127.0.0.1:9095'], null, callback);
+}
+
+/**
+ * Starts ignite node with default config
+ *
+ * @param {string} secretKey Secret key
+ * @param {Ignition~onStart} callback Called on connect
+ */
+TestUtils.startIgniteNodeWithKey = function(secretKey, callback) {
+  var Apache = require(TestUtils.scriptPath());
+  var Ignition = Apache.Ignition;
+  Ignition.start(['127.0.0.1:9095'], secretKey, callback);
 }
 
 exports.TestUtils = TestUtils;