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/03/04 14:26:41 UTC

[06/22] incubator-ignite git commit: #ignite-51: IgniteTxEntry implements Message: GridNearTxPrepareResponseOwnedValue to upper level.

#ignite-51: IgniteTxEntry implements Message: GridNearTxPrepareResponseOwnedValue to upper level.


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

Branch: refs/heads/ignite-51
Commit: f138b5760d081a9521ea28e0437b8f33a5781d6f
Parents: add068f
Author: ivasilinets <iv...@gridgain.com>
Authored: Tue Mar 3 18:46:58 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Tue Mar 3 18:46:58 2015 +0300

----------------------------------------------------------------------
 .../communication/GridIoMessageFactory.java     |   2 +-
 .../near/GridNearTxPrepareFuture.java           |   6 +-
 .../near/GridNearTxPrepareResponse.java         | 147 ++----------------
 .../GridNearTxPrepareResponseOwnedValue.java    | 155 +++++++++++++++++++
 4 files changed, 168 insertions(+), 142 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f138b576/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 5b0f7dc..2126d86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -545,7 +545,7 @@ public class GridIoMessageFactory implements MessageFactory {
                 break;
 
             case 99:
-                msg = new GridNearTxPrepareResponse.OwnedValue();
+                msg = new GridNearTxPrepareResponseOwnedValue();
 
                 break;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f138b576/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index 0f39b5e..dc2c11d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -934,7 +934,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
                 else {
                     assert F.isEmpty(res.invalidPartitions());
 
-                    for (Map.Entry<IgniteTxKey, GridNearTxPrepareResponse.OwnedValue> entry : res.ownedValues().entrySet()) {
+                    for (Map.Entry<IgniteTxKey, GridNearTxPrepareResponseOwnedValue> entry : res.ownedValues().entrySet()) {
                         IgniteTxEntry txEntry = tx.entry(entry.getKey());
 
                         assert txEntry != null;
@@ -946,7 +946,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
                                 if (cacheCtx.isNear()) {
                                     GridNearCacheEntry nearEntry = (GridNearCacheEntry)txEntry.cached();
 
-                                    GridNearTxPrepareResponse.OwnedValue tup = entry.getValue();
+                                    GridNearTxPrepareResponseOwnedValue tup = entry.getValue();
 
                                     nearEntry.resetFromPrimary(tup.cacheObject(), tx.xidVersion(),
                                         tup.version(), m.node().id());
@@ -954,7 +954,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
                                 else if (txEntry.cached().detached()) {
                                     GridDhtDetachedCacheEntry detachedEntry = (GridDhtDetachedCacheEntry)txEntry.cached();
 
-                                    GridNearTxPrepareResponse.OwnedValue tup = entry.getValue();
+                                    GridNearTxPrepareResponseOwnedValue tup = entry.getValue();
 
                                     detachedEntry.resetFromPrimary(tup.cacheObject(), tx.xidVersion());
                                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f138b576/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
index beb2015..d6819bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
@@ -61,7 +61,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
     /** Map of owned values to set on near node. */
     @GridToStringInclude
     @GridDirectTransient
-    private Map<IgniteTxKey, OwnedValue> ownedVals;
+    private Map<IgniteTxKey, GridNearTxPrepareResponseOwnedValue> ownedVals;
 
     /** OwnedVals' keys for marshalling. */
     @GridToStringExclude
@@ -70,8 +70,8 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
     /** OwnedVals' values for marshalling. */
     @GridToStringExclude
-    @GridDirectCollection(OwnedValue.class)
-    private Collection<OwnedValue> ownedValVals;
+    @GridDirectCollection(GridNearTxPrepareResponseOwnedValue.class)
+    private Collection<GridNearTxPrepareResponseOwnedValue> ownedValVals;
 
     /** Cache return value. */
     @GridDirectTransient
@@ -174,7 +174,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         if (ownedVals == null)
             ownedVals = new HashMap<>();
 
-        OwnedValue oVal = new OwnedValue(ver, val);
+        GridNearTxPrepareResponseOwnedValue oVal = new GridNearTxPrepareResponseOwnedValue(ver, val);
 
         ownedVals.put(key, oVal);
     }
@@ -182,9 +182,9 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
     /**
      * @return Owned values map.
      */
-    public Map<IgniteTxKey, OwnedValue> ownedValues() {
+    public Map<IgniteTxKey, GridNearTxPrepareResponseOwnedValue> ownedValues() {
         return ownedVals == null ?
-            Collections.<IgniteTxKey, OwnedValue>emptyMap() :
+            Collections.<IgniteTxKey, GridNearTxPrepareResponseOwnedValue>emptyMap() :
             Collections.unmodifiableMap(ownedVals);
     }
 
@@ -234,7 +234,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
             ownedValVals = ownedVals.values();
 
-            for (Map.Entry<IgniteTxKey, OwnedValue> entry : ownedVals.entrySet()) {
+            for (Map.Entry<IgniteTxKey, GridNearTxPrepareResponseOwnedValue> entry : ownedVals.entrySet()) {
                 GridCacheContext cacheCtx = ctx.cacheContext(entry.getKey().cacheId());
 
                 entry.getKey().prepareMarshal(cacheCtx);
@@ -266,14 +266,14 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
             Iterator<IgniteTxKey> keyIter = ownedValKeys.iterator();
 
-            Iterator<OwnedValue> valueIter = ownedValVals.iterator();
+            Iterator<GridNearTxPrepareResponseOwnedValue> valueIter = ownedValVals.iterator();
 
             while (keyIter.hasNext()) {
                 IgniteTxKey key = keyIter.next();
 
                 GridCacheContext cctx = ctx.cacheContext(key.cacheId());
 
-                OwnedValue value = valueIter.next();
+                GridNearTxPrepareResponseOwnedValue value = valueIter.next();
 
                 key.finishUnmarshal(cctx, ldr);
 
@@ -472,133 +472,4 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         return S.toString(GridNearTxPrepareResponse.class, this, "super", super.toString());
     }
 
-    /**
-     * Message for owned values to set on near node.
-     */
-    public static class OwnedValue implements Message {
-        /** Cache version. */
-        private GridCacheVersion vers;
-
-        /** Cache object. */
-        private CacheObject obj;
-
-        /** */
-        public OwnedValue() {
-            // No-op.
-        }
-
-        /**
-         * @param vers Cache version.
-         * @param obj Cache object.
-         */
-        OwnedValue(GridCacheVersion vers, CacheObject obj) {
-            this.vers = vers;
-            this.obj = obj;
-        }
-
-        /**
-         * @return Cache version.
-         */
-        public GridCacheVersion version() {
-            return vers;
-        }
-
-        /**
-         * @return Cache object.
-         */
-        public CacheObject cacheObject() {
-            return obj;
-        }
-
-        /**
-         * This method is called before the whole message is sent
-         * and is responsible for pre-marshalling state.
-         *
-         * @param ctx Cache object context.
-         * @throws IgniteCheckedException If failed.
-         */
-        public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
-            if (obj != null)
-                obj.prepareMarshal(ctx);
-        }
-
-        /**
-         * This method is called after the whole message is recived
-         * and is responsible for unmarshalling state.
-         *
-         * @param ctx Context.
-         * @param ldr Class loader.
-         * @throws IgniteCheckedException If failed.
-         */
-        public void finishUnmarshal(GridCacheContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-            if (obj != null)
-                obj.finishUnmarshal(ctx, ldr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-            writer.setBuffer(buf);
-
-            if (!writer.isHeaderWritten()) {
-                if (!writer.writeHeader(directType(), fieldsCount()))
-                    return false;
-
-                writer.onHeaderWritten();
-            }
-
-            switch (writer.state()) {
-                case 0:
-                    if (!writer.writeMessage("vers", vers))
-                        return false;
-
-                    writer.incrementState();
-
-                case 1:
-                    if (!writer.writeMessage("obj", obj))
-                        return false;
-
-                    writer.incrementState();
-            }
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-            reader.setBuffer(buf);
-
-            if (!reader.beforeMessageRead())
-                return false;
-
-            switch (reader.state()) {
-                case 0:
-                    vers = reader.readMessage("vers");
-
-                    if (!reader.isLastRead())
-                        return false;
-
-                    reader.incrementState();
-
-                case 1:
-                    obj = reader.readMessage("obj");
-
-                    if (!reader.isLastRead())
-                        return false;
-
-                    reader.incrementState();
-            }
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte directType() {
-            return 99;
-        }
-
-        /** {@inheritDoc} */
-        @Override public byte fieldsCount() {
-            return 2;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f138b576/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponseOwnedValue.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponseOwnedValue.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponseOwnedValue.java
new file mode 100644
index 0000000..b833f78
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponseOwnedValue.java
@@ -0,0 +1,155 @@
+/*
+ * 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.processors.cache.distributed.near;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+
+import java.nio.*;
+
+/**
+ * Message for owned values to set on near node.
+ */
+public class GridNearTxPrepareResponseOwnedValue implements Message {
+    /** Cache version. */
+    private GridCacheVersion vers;
+
+    /** Cache object. */
+    private CacheObject obj;
+
+    /** */
+    public GridNearTxPrepareResponseOwnedValue() {
+        // No-op.
+    }
+
+    /**
+     * @param vers Cache version.
+     * @param obj Cache object.
+     */
+    GridNearTxPrepareResponseOwnedValue(GridCacheVersion vers, CacheObject obj) {
+        this.vers = vers;
+        this.obj = obj;
+    }
+
+    /**
+     * @return Cache version.
+     */
+    public GridCacheVersion version() {
+        return vers;
+    }
+
+    /**
+     * @return Cache object.
+     */
+    public CacheObject cacheObject() {
+        return obj;
+    }
+
+    /**
+     * This method is called before the whole message is sent
+     * and is responsible for pre-marshalling state.
+     *
+     * @param ctx Cache object context.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
+        if (obj != null)
+            obj.prepareMarshal(ctx);
+    }
+
+    /**
+     * This method is called after the whole message is recived
+     * and is responsible for unmarshalling state.
+     *
+     * @param ctx Context.
+     * @param ldr Class loader.
+     * @throws org.apache.ignite.IgniteCheckedException If failed.
+     */
+    public void finishUnmarshal(GridCacheContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        if (obj != null)
+            obj.finishUnmarshal(ctx, ldr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeMessage("vers", vers))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeMessage("obj", obj))
+                    return false;
+
+                writer.incrementState();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                vers = reader.readMessage("vers");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                obj = reader.readMessage("obj");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return 99;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 2;
+    }
+}