You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by iv...@apache.org on 2015/07/27 07:48:32 UTC

incubator-ignite git commit: #ignite-961-permissions: init commit.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-961-permissions [created] 7885f1f5f


#ignite-961-permissions: init commit.


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

Branch: refs/heads/ignite-961-permissions
Commit: 7885f1f5fa66100e5e5faeeff684f10921175876
Parents: 0341759
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Jul 27 08:48:18 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Jul 27 08:48:18 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    |  6 +++++
 .../processors/cache/GridCacheMessage.java      |  2 +-
 .../GridCachePartitionExchangeManager.java      |  2 +-
 .../processors/cache/GridCacheProcessor.java    | 21 +++++++++++++++-
 .../dht/GridDhtPartitionTopologyImpl.java       |  3 +++
 .../GridDhtPartitionsExchangeFuture.java        | 23 +++++++++++------
 .../preloader/GridDhtPartitionsFullMessage.java | 26 +++++++++++++++++++-
 7 files changed, 72 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 84e4dc2..b8ebbc5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -426,6 +426,12 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             break;
 
+            case 46: {
+                GridDhtPartitionsFullMessage req = (GridDhtPartitionsFullMessage)msg;
+
+                ctx.cache().
+            }
+
             case 49: {
                 GridNearGetRequest req = (GridNearGetRequest)msg;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 8ebce35..b786b3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -60,7 +60,7 @@ public abstract class GridCacheMessage implements Message {
 
     /** */
     @GridDirectTransient
-    private IgniteCheckedException err;
+    protected IgniteCheckedException err;
 
     /** */
     @GridDirectTransient

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index c26f5c3..79728df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -920,7 +920,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
                             // Finished future should reply only to sender client node.
-                            exchFut.onReceive(node.id(), msg);
+                            exchFut.onReceive(node.id(), msg, fut.error());
                         }
                     });
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index f5ccaec..9f9a698 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -55,6 +55,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
 
@@ -1017,6 +1018,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      */
     @SuppressWarnings({"TypeMayBeWeakened", "unchecked"})
     private void startCache(GridCacheAdapter<?, ?> cache) throws IgniteCheckedException {
+        //TODO:
+        if (!systemCache(cache.name()))
+            checkSecurityPermissions();
+
         GridCacheContext<?, ?> cacheCtx = cache.context();
 
         ctx.query().onCacheStart(cacheCtx);
@@ -1687,7 +1692,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             GridCacheContext<?, ?> cacheCtx = cache.context();
 
             if (F.eq(cacheCtx.startTopologyVersion(), topVer)) {
-                cacheCtx.preloader().onInitialExchangeComplete(err);
+                if (cacheCtx.preloader() != null)
+                    cacheCtx.preloader().onInitialExchangeComplete(err);
 
                 String masked = maskNull(cacheCtx.name());
 
@@ -2093,6 +2099,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         boolean failIfExists,
         boolean failIfNotStarted
     ) {
+        checkSecurityPermissions();
         checkEmptyTransactions();
 
         DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
@@ -2186,6 +2193,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when cache is destroyed.
      */
     public IgniteInternalFuture<?> dynamicDestroyCache(String cacheName) {
+        checkSecurityPermissions();
         checkEmptyTransactions();
 
         DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
@@ -2206,6 +2214,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (proxy == null || proxy.proxyClosed())
             return new GridFinishedFuture<>(); // No-op.
 
+        checkSecurityPermissions();
         checkEmptyTransactions();
 
         DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
@@ -3230,6 +3239,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @throws org.apache.ignite.plugin.security.SecurityException If failed.
+     */
+    private void checkSecurityPermissions() throws org.apache.ignite.plugin.security.SecurityException {
+        if (!ctx.security().enabled() || ctx.clientNode())
+            return;
+
+        ctx.security().authorize(null, SecurityPermission.ADMIN_CACHE, null);
+    }
+
+    /**
      * @param val Object to check.
      * @throws IgniteCheckedException If validation failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index facf329..1fc482f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -767,6 +767,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         lock.readLock().lock();
 
         try {
+            if (node2part == null)
+                return new GridDhtPartitionFullMap();
+
             assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
                 ", cache=" + cctx.name() +
                 ", started=" + cctx.started() +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 3664220..bb59af3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -919,12 +919,14 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param id ID.
      * @throws IgniteCheckedException If failed.
      */
-    private void sendAllPartitions(Collection<? extends ClusterNode> nodes, GridDhtPartitionExchangeId id)
+    private void sendAllPartitions(Collection<? extends ClusterNode> nodes, GridDhtPartitionExchangeId id,
+        Throwable error)
         throws IgniteCheckedException {
         GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(id,
             lastVer.get(),
             id.topologyVersion());
-
+        if (error != null)
+            m.onClassError(new IgniteCheckedException(error));
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
             if (!cacheCtx.isLocal()) {
                 AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion();
@@ -974,7 +976,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      */
     private boolean spreadPartitions() {
         try {
-            sendAllPartitions(rmtNodes, exchId);
+            sendAllPartitions(rmtNodes, exchId, null);
 
             return true;
         }
@@ -1072,6 +1074,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param msg Single partition info.
      */
     public void onReceive(final UUID nodeId, final GridDhtPartitionsSingleMessage msg) {
+        onReceive(nodeId, msg, null);
+    }
+    /**
+     * @param nodeId Sender node id.
+     * @param msg Single partition info.
+     */
+    public void onReceive(final UUID nodeId, final GridDhtPartitionsSingleMessage msg, Throwable error) {
         assert msg != null;
 
         assert msg.exchangeId().equals(exchId);
@@ -1093,7 +1102,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 log.debug("Received message for finished future (will reply only to sender) [msg=" + msg +
                     ", fut=" + this + ']');
 
-            sendAllPartitions(nodeId, cctx.gridConfig().getNetworkSendRetryCount());
+            sendAllPartitions(nodeId, cctx.gridConfig().getNetworkSendRetryCount(), error);
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
@@ -1153,12 +1162,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param nodeId Node ID.
      * @param retryCnt Number of retries.
      */
-    private void sendAllPartitions(final UUID nodeId, final int retryCnt) {
+    private void sendAllPartitions(final UUID nodeId, final int retryCnt, Throwable error) {
         ClusterNode n = cctx.node(nodeId);
 
         try {
             if (n != null)
-                sendAllPartitions(F.asList(n), exchId);
+                sendAllPartitions(F.asList(n), exchId, error);
         }
         catch (IgniteCheckedException e) {
             if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) {
@@ -1176,7 +1185,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                 cctx.time().addTimeoutObject(new GridTimeoutObjectAdapter(timeout) {
                     @Override public void onTimeout() {
-                        sendAllPartitions(nodeId, retryCnt - 1);
+                        sendAllPartitions(nodeId, retryCnt - 1, null);
                     }
                 });
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7885f1f5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index 73794ae..ddba5c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -46,6 +46,9 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     /** */
     private byte[] partsBytes;
 
+    /** */
+    private byte[] errBytes;
+
     /** Topology version. */
     private AffinityTopologyVersion topVer;
 
@@ -95,6 +98,9 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
         if (parts != null)
             partsBytes = ctx.marshaller().marshal(parts);
+
+        if (err != null)
+            errBytes = ctx.marshaller().marshal(err);
     }
 
     /**
@@ -117,6 +123,9 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
         if (partsBytes != null)
             parts = ctx.marshaller().unmarshal(partsBytes, ldr);
+
+        if (errBytes != null)
+            err = ctx.marshaller().unmarshal(errBytes, ldr);
     }
 
     /** {@inheritDoc} */
@@ -146,6 +155,13 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 writer.incrementState();
 
+            case 7:
+                if (!writer.writeByteArray("errBytes", errBytes))
+                    return false;
+
+                writer.incrementState();
+
+
         }
 
         return true;
@@ -178,6 +194,14 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
                 reader.incrementState();
 
+            case 7:
+                errBytes = reader.readByteArray("errBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
         return true;
@@ -190,7 +214,7 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 7;
+        return 8;
     }
 
     /** {@inheritDoc} */