You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/02/25 13:31:39 UTC

[40/51] [abbrv] ignite git commit: Review.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c814ae3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponseInterface.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponseInterface.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponseInterface.java
new file mode 100644
index 0000000..f521f7d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponseInterface.java
@@ -0,0 +1,211 @@
+/*
+ * 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.dht.atomic;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.cache.CacheObject;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheReturn;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.jetbrains.annotations.Nullable;
+import java.util.Collection;
+import java.util.List;
+import java.util.UUID;
+
+/**
+ * Base interface for near atomic update responses.
+ */
+public interface GridNearAtomicUpdateResponseInterface extends Message {
+
+    /**
+     * Gets message lookup index. See {@link GridCacheMessage#lookupIndex()}.
+     *
+     * @return Message lookup index.
+     */
+    int lookupIndex();
+
+    /**
+     * @return Mapped node ID.
+     */
+    UUID nodeId();
+
+    /**
+     * @param nodeId Node ID.
+     */
+    void nodeId(UUID nodeId);
+
+    /**
+     * @return Future version.
+     */
+    GridCacheVersion futureVersion();
+
+    /**
+     * Sets update error.
+     *
+     * @param err Error.
+     */
+    void error(IgniteCheckedException err);
+
+    /**
+     * @return Error, if any.
+     */
+    IgniteCheckedException error();
+
+    /**
+     * @return Collection of failed keys.
+     */
+    Collection<KeyCacheObject> failedKeys();
+
+    /**
+     * @return Return value.
+     */
+    GridCacheReturn returnValue();
+
+    /**
+     * @param ret Return value.
+     */
+    @SuppressWarnings("unchecked") void returnValue(GridCacheReturn ret);
+
+    /**
+     * @param remapKeys Remap keys.
+     */
+    void remapKeys(List<KeyCacheObject> remapKeys);
+
+    /**
+     * @return Remap keys.
+     */
+    Collection<KeyCacheObject> remapKeys();
+
+    /**
+     * Adds value to be put in near cache on originating node.
+     *
+     * @param keyIdx Key index.
+     * @param val Value.
+     * @param ttl TTL for near cache update.
+     * @param expireTime Expire time for near cache update.
+     */
+    void addNearValue(int keyIdx,
+        @Nullable CacheObject val,
+        long ttl,
+        long expireTime);
+
+    /**
+     * @param keyIdx Key index.
+     * @param ttl TTL for near cache update.
+     * @param expireTime Expire time for near cache update.
+     */
+    @SuppressWarnings("ForLoopReplaceableByForEach") void addNearTtl(int keyIdx, long ttl, long expireTime);
+
+    /**
+     * @param idx Index.
+     * @return Expire time for near cache update.
+     */
+    long nearExpireTime(int idx);
+
+    /**
+     * @param idx Index.
+     * @return TTL for near cache update.
+     */
+    long nearTtl(int idx);
+
+    /**
+     * @param nearVer Version generated on primary node to be used for originating node's near cache update.
+     */
+    void nearVersion(GridCacheVersion nearVer);
+
+    /**
+     * @return Version generated on primary node to be used for originating node's near cache update.
+     */
+    GridCacheVersion nearVersion();
+
+    /**
+     * @param keyIdx Index of key for which update was skipped
+     */
+    void addSkippedIndex(int keyIdx);
+
+    /**
+     * @return Indexes of keys for which update was skipped
+     */
+    @Nullable List<Integer> skippedIndexes();
+
+    /**
+     * @return Indexes of keys for which values were generated on primary node.
+     */
+    @Nullable List<Integer> nearValuesIndexes();
+
+    /**
+     * @param idx Index.
+     * @return Value generated on primary node which should be put to originating node's near cache.
+     */
+    @Nullable CacheObject nearValue(int idx);
+
+    /**
+     * Adds key to collection of failed keys.
+     *
+     * @param key Key to add.
+     * @param e Error cause.
+     */
+    void addFailedKey(KeyCacheObject key, Throwable e);
+
+    /**
+     * Adds keys to collection of failed keys.
+     *
+     * @param keys Key to add.
+     * @param e Error cause.
+     */
+    void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e);
+
+    /**
+     * Adds keys to collection of failed keys.
+     *
+     * @param keys Key to add.
+     * @param e Error cause.
+     * @param ctx Context.
+     */
+    void addFailedKeys(Collection<KeyCacheObject> keys, Throwable e, GridCacheContext ctx);
+
+    /**
+     * This method is called before the whole message is serialized
+     * and is responsible for pre-marshalling state.
+     *
+     * @param ctx Cache context.
+     * @throws IgniteCheckedException If failed.
+     */
+    void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException;
+
+    /**
+     * This method is called after the message is deserialized and is responsible for
+     * unmarshalling state marshalled in {@link #prepareMarshal(GridCacheSharedContext)} method.
+     *
+     * @param ctx Context.
+     * @param ldr Class loader.
+     * @throws IgniteCheckedException If failed.
+     */
+    void finishUnmarshal(GridCacheSharedContext ctx, ClassLoader ldr) throws IgniteCheckedException;
+
+    /**
+     *  Deployment enabled flag indicates whether deployment info has to be added to this message.
+     *
+     * @return {@code true} or if deployment info must be added to the the message, {@code false} otherwise.
+     */
+    boolean addDeploymentInfo();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c814ae3a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 4f133e1..efe8347 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -33,10 +33,10 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequestInterface;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponseInterface;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequestInterface;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponseInterface;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -125,8 +125,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
      * @param res Update response.
      */
     public void processNearAtomicUpdateResponse(
-        GridNearAtomicUpdateRequest req,
-        GridNearAtomicUpdateResponse res
+        GridNearAtomicUpdateRequestInterface req,
+        GridNearAtomicUpdateResponseInterface res
     ) {
         if (F.size(res.failedKeys()) == req.keys().size())
             return;
@@ -300,8 +300,8 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
      */
     public void processDhtAtomicUpdateRequest(
         UUID nodeId,
-        GridDhtAtomicUpdateRequest req,
-        GridDhtAtomicUpdateResponse res
+        GridDhtAtomicUpdateRequestInterface req,
+        GridDhtAtomicUpdateResponseInterface res
     ) {
         GridCacheVersion ver = req.writeVersion();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c814ae3a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
index 5509496..4c011db 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -27,7 +27,7 @@ import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteSet;
 import org.apache.ignite.configuration.CollectionConfiguration;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicMultipleUpdateResponse;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponseInterface;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareResponse;
 import org.apache.ignite.testframework.GridTestUtils;
 
@@ -458,7 +458,7 @@ public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectA
         BlockTpcCommunicationSpi commSpi = commSpi(srv);
 
         if (colCfg.getAtomicityMode() == ATOMIC)
-            commSpi.blockMessage(GridNearAtomicUpdateResponse.class);
+            commSpi.blockMessage(GridNearAtomicUpdateResponseInterface.class);
         else
             commSpi.blockMessage(GridNearTxPrepareResponse.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c814ae3a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 49686fc..4733e19 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -63,7 +63,7 @@ import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicMultipleUpdateRequest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateRequestInterface;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheAdapter;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearCacheEntry;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest;
@@ -412,7 +412,7 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
         assertEquals(3, msgs.size());
 
         for (Object msg : msgs)
-            assertTrue(((GridNearAtomicUpdateRequest)msg).clientRequest());
+            assertTrue(((GridNearAtomicUpdateRequestInterface)msg).clientRequest());
 
         map.put(primaryKey(ignite0.cache(null)), 3);
         map.put(primaryKey(ignite1.cache(null)), 4);