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/23 15:52:44 UTC

[01/33] incubator-ignite git commit: IGNITE-1034 - Drop slow clients.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-950 5fe19ff97 -> 5749b068d


IGNITE-1034 - Drop slow clients.


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

Branch: refs/heads/ignite-950
Commit: 44bbeceae657058eee322b13e339966c625802db
Parents: ad0a026
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 18 19:06:47 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 18 19:06:47 2015 -0700

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   4 +
 .../discovery/GridDiscoveryManager.java         |  15 +++
 .../ignite/internal/util/nio/GridNioServer.java |  51 ++++++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   5 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   5 +
 .../communication/tcp/TcpCommunicationSpi.java  |  73 +++++++++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   5 +
 .../IgniteSlowClientDetectionSelfTest.java      | 124 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   5 +
 9 files changed, 287 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index bea4256..885d52c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -484,6 +484,10 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.discovery().tryFailNode(nodeId);
                     }
 
+                    @Override public void failNode(UUID nodeId) {
+                        ctx.discovery().failNode(nodeId);
+                    }
+
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
                         ctx.timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 464110c..717cdf3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -1502,6 +1502,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @param nodeId Node ID to fail.
+     */
+    public void failNode(UUID nodeId) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            getSpi().failNode(nodeId);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 88fad71..b9d246a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -116,6 +116,10 @@ public class GridNioServer<T> {
     @SuppressWarnings("UnusedDeclaration")
     private boolean skipWrite;
 
+    /** For test purposes only. */
+    @SuppressWarnings("UnusedDeclaration")
+    private boolean skipRead;
+
     /** Local address. */
     private final InetSocketAddress locAddr;
 
@@ -145,6 +149,9 @@ public class GridNioServer<T> {
     @GridToStringExclude
     private IgnitePredicate<Message> skipRecoveryPred;
 
+    /** Optional listener to monitor outbound message queue size. */
+    private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+
     /** Static initializer ensures single-threaded execution of workaround. */
     static {
         // This is a workaround for JDK bug (NPE in Selector.open()).
@@ -174,6 +181,7 @@ public class GridNioServer<T> {
      * @param metricsLsnr Metrics listener.
      * @param formatter Message formatter.
      * @param skipRecoveryPred Skip recovery predicate.
+     * @param msgQueueLsnr Message queue size listener.
      * @param filters Filters for this server.
      * @throws IgniteCheckedException If failed.
      */
@@ -195,6 +203,7 @@ public class GridNioServer<T> {
         GridNioMetricsListener metricsLsnr,
         MessageFormatter formatter,
         IgnitePredicate<Message> skipRecoveryPred,
+        IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         A.notNull(addr, "addr");
@@ -215,6 +224,7 @@ public class GridNioServer<T> {
         this.sockRcvBuf = sockRcvBuf;
         this.sockSndBuf = sockSndBuf;
         this.sndQueueLimit = sndQueueLimit;
+        this.msgQueueLsnr = msgQueueLsnr;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -385,6 +395,11 @@ public class GridNioServer<T> {
         else if (msgCnt == 1)
             // Change from 0 to 1 means that worker thread should be waken up.
             clientWorkers.get(ses.selectorIndex()).offer(fut);
+
+        IgniteBiInClosure<GridNioSession, Integer> lsnr0 = msgQueueLsnr;
+
+        if (lsnr0 != null)
+            lsnr0.apply(ses, msgCnt);
     }
 
     /**
@@ -634,6 +649,17 @@ public class GridNioServer<T> {
         * @throws IOException If key read failed.
         */
         @Override protected void processRead(SelectionKey key) throws IOException {
+            if (skipRead) {
+                try {
+                    U.sleep(50);
+                }
+                catch (IgniteInterruptedCheckedException ignored) {
+                    U.warn(log, "Sleep has been interrupted.");
+                }
+
+                return;
+            }
+
             ReadableByteChannel sockCh = (ReadableByteChannel)key.channel();
 
             final GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
@@ -775,6 +801,17 @@ public class GridNioServer<T> {
          * @throws IOException If key read failed.
          */
         @Override protected void processRead(SelectionKey key) throws IOException {
+            if (skipRead) {
+                try {
+                    U.sleep(50);
+                }
+                catch (IgniteInterruptedCheckedException ignored) {
+                    U.warn(log, "Sleep has been interrupted.");
+                }
+
+                return;
+            }
+
             ReadableByteChannel sockCh = (ReadableByteChannel)key.channel();
 
             final GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
@@ -2108,6 +2145,9 @@ public class GridNioServer<T> {
         /** Skip recovery predicate. */
         private IgnitePredicate<Message> skipRecoveryPred;
 
+        /** Message queue size listener. */
+        private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+
         /**
          * Finishes building the instance.
          *
@@ -2133,6 +2173,7 @@ public class GridNioServer<T> {
                 metricsLsnr,
                 formatter,
                 skipRecoveryPred,
+                msgQueueLsnr,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -2345,5 +2386,15 @@ public class GridNioServer<T> {
 
             return this;
         }
+
+        /**
+         * @param msgQueueLsnr Message queue size listener.
+         * @return Instance of this builder for chaining.
+         */
+        public Builder<T> messageQueueSizeListener(IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr) {
+            this.msgQueueLsnr = msgQueueLsnr;
+
+            return this;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 476f8a8..18191a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -759,6 +759,11 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
+        @Override public void failNode(UUID nodeId) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
             assert ignite instanceof IgniteKernal : ignite;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index f83326c..a655a73 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -312,6 +312,11 @@ public interface IgniteSpiContext {
     public boolean tryFailNode(UUID nodeId);
 
     /**
+     * @param nodeId Node ID.
+     */
+    public void failNode(UUID nodeId);
+
+    /**
      * @param c Timeout object.
      */
     public void addTimeoutObject(IgniteSpiTimeoutObject c);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 87d5b65..538e9a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.ipc.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
@@ -610,6 +611,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @LoggerResource
     private IgniteLogger log;
 
+    /** Ignite. */
+    @IgniteInstanceResource
+    @GridToStringExclude
+    private Ignite ignite;
+
     /** Local IP address. */
     private String locAddr;
 
@@ -653,6 +659,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Message queue limit. */
     private int msgQueueLimit = DFLT_MSG_QUEUE_LIMIT;
 
+    /** Slow client queue limit. */
+    private int slowClientQueueLimit;
+
     /** Min buffered message count. */
     private int minBufferedMsgCnt = Integer.getInteger(IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT, 512);
 
@@ -1145,6 +1154,30 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    public int getSlowClientQueueLimit() {
+        return slowClientQueueLimit;
+    }
+
+    /**
+     * Sets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @param slowClientQueueLimit Slow cilent queue limit.
+     */
+    public void setSlowClientQueueLimit(int slowClientQueueLimit) {
+        this.slowClientQueueLimit = slowClientQueueLimit;
+    }
+
+    /**
      * Sets the minimum number of messages for this SPI, that are buffered
      * prior to sending.
      * <p>
@@ -1315,6 +1348,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
 
+        if (slowClientQueueLimit > 0 && msgQueueLimit > 0) {
+            if (slowClientQueueLimit >= msgQueueLimit) {
+                U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit. " +
+                    "Slow client queue limit will have no effect.");
+            }
+        }
+
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
         if (shmemSrv != null) {
@@ -1425,6 +1465,15 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
+                IgniteBiInClosure<GridNioSession, Integer> queueSizeMonitor =
+                    !ignite.configuration().isClientMode() && slowClientQueueLimit > 0 ?
+                    new CI2<GridNioSession, Integer>() {
+                        @Override public void apply(GridNioSession ses, Integer qSize) {
+                            checkClientQueueSize(ses, qSize);
+                        }
+                    } :
+                    null;
+
                 GridNioServer<Message> srvr =
                     GridNioServer.<Message>builder()
                         .address(locHost)
@@ -1446,6 +1495,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             new GridConnectionBytesVerifyFilter(log))
                         .messageFormatter(msgFormatter)
                         .skipRecoveryPredicate(skipRecoveryPred)
+                        .messageQueueSizeListener(queueSizeMonitor)
                         .build();
 
                 boundTcpPort = port;
@@ -1860,6 +1910,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Checks client message queue size and initiates client drop if message queue size exceeds the configured limit.
+     *
+     * @param ses Node communication session.
+     * @param msgQueueSize Message queue size.
+     */
+    private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) {
+        if (slowClientQueueLimit > 0 && msgQueueSize > slowClientQueueLimit) {
+            UUID id = ses.meta(NODE_ID_META);
+
+            if (id != null) {
+                ClusterNode node = getSpiContext().node(id);
+
+                if (node != null && node.isClient()) {
+                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+
+                    getSpiContext().failNode(id);
+                }
+            }
+        }
+    }
+
+    /**
      * Establish TCP connection to remote node and returns client.
      *
      * @param node Remote node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index b743a1a..8eb82ac 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3402,6 +3402,11 @@ class ServerImpl extends TcpDiscoveryImpl {
                     failedNodes.remove(node);
 
                     leavingNodes.remove(node);
+
+                    ClientMessageWorker worker = clientMsgWorkers.remove(node.id());
+
+                    if (worker != null)
+                        worker.interrupt();
                 }
 
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
new file mode 100644
index 0000000..09b4215
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -0,0 +1,124 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.event.*;
+
+/**
+ *
+ */
+public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
+
+    public static final String PARTITIONED = "partitioned";
+
+    /**
+     * @return Node count.
+     */
+    private int nodeCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
+            cfg.setClientMode(true);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSlowClientQueueLimit(50);
+        commSpi.setSharedMemoryPort(-1);
+        commSpi.setIdleConnectionTimeout(300_000);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(nodeCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSlowClient() throws Exception {
+        final IgniteEx slowClient = grid(nodeCount() - 1);
+
+        assertTrue(slowClient.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
+
+        IgniteEx client0 = grid(nodeCount() - 2);
+
+        assertTrue(client0.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> cache0 = client0.getOrCreateCache(PARTITIONED);
+
+        cache.query(new ContinuousQuery<>().setLocalListener(new Listener()));
+
+        for (int i = 0; i < 100; i++)
+            cache0.put(0, i);
+
+        GridIoManager ioMgr = slowClient.context().io();
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)((Object[])U.field(ioMgr, "spis"))[0];
+
+        GridNioServer nioSrvr = U.field(commSpi, "nioSrvr");
+
+        GridTestUtils.setFieldValue(nioSrvr, "skipRead", true);
+
+        // Initiate messages for client.
+        for (int i = 0; i < 100; i++)
+            cache0.put(0, new byte[10 * 1024]);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
+            }
+        }, getTestTimeout());
+    }
+
+    private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
+            System.out.println(">>>> Received update: " + iterable);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index 21f9424..c20ff2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -502,6 +502,11 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
         // No-op.
     }


[32/33] incubator-ignite git commit: ignite-950: implementing marshal aware

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStreamExt.java
new file mode 100644
index 0000000..fc0daa6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStreamExt.java
@@ -0,0 +1,152 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.io.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerExt.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public class OptimizedObjectOutputStreamExt extends OptimizedObjectOutputStream {
+    /**
+     * Constructor.
+     *
+     * @param out Output stream.
+     * @throws IOException In case of error.
+     */
+    protected OptimizedObjectOutputStreamExt(GridDataOutput out) throws IOException {
+        super(out);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void writeFieldType(byte type) throws IOException {
+        out.writeByte(type);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Footer createFooter(Class<?> cls) {
+        if (!ctx.isSystemType(cls.getName()) && (OptimizedMarshalAware.class.isAssignableFrom(cls) ||
+            (metaHandler != null && metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null)))
+            return new FooterImpl();
+        else
+            return null;
+    }
+
+    /**
+     *
+     */
+    private class FooterImpl implements OptimizedObjectOutputStream.Footer {
+        /** */
+        private ArrayList<Integer> data;
+
+        /** */
+        private ArrayList<Integer> fields;
+
+        /** */
+        private HashMap<Integer, GridHandleTable.ObjectInfo> handles;
+
+        /** */
+        private boolean hasHandles;
+
+        /** {@inheritDoc} */
+        @Override public void fields(OptimizedClassDescriptor.Fields fields) {
+            if (fields.fieldsIndexingSupported()) {
+                data = new ArrayList<>();
+                this.fields = new ArrayList<>();
+            }
+            else
+                data = null;
+        }
+
+        /** {@inheritDoc} */
+        public void put(int fieldId, OptimizedFieldType fieldType, int len) {
+            if (data == null)
+                return;
+
+            if (fieldType == OptimizedFieldType.OTHER) {
+                data.add(len);
+                fields.add(fieldId);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void putHandle(int fieldId, GridHandleTable.ObjectInfo objInfo) {
+            if (data == null)
+                return;
+
+            if (!hasHandles) {
+                hasHandles = true;
+                handles = new HashMap<>();
+            }
+
+            handles.put(fieldId, objInfo);
+
+            // length of handle fields is 5 bytes.
+            put(fieldId, OptimizedFieldType.OTHER, 5);
+        }
+
+        /** {@inheritDoc} */
+        public void write() throws IOException {
+            if (data == null)
+                writeInt(EMPTY_FOOTER);
+            else {
+                int bodyEnd = out.offset();
+
+                // +4 - 2 bytes for footer len at the beginning, 2 bytes for footer len at the end.
+                short footerLen = (short)(data.size() * 4 + 4);
+
+                if (hasHandles)
+                    footerLen += handles.size() * 8;
+
+                writeShort(footerLen);
+
+                if (hasHandles) {
+                    for (int i = 0; i < data.size(); i++) {
+                        GridHandleTable.ObjectInfo objInfo = handles.get(fields.get(i));
+
+                        if (objInfo == null)
+                            writeInt(data.get(i) & ~FOOTER_BODY_IS_HANDLE_MASK);
+                        else {
+                            writeInt(data.get(i) | FOOTER_BODY_IS_HANDLE_MASK);
+                            writeInt(objInfo.position());
+
+                            if (objInfo.length() == 0)
+                                // field refers to its own object that hasn't set total length yet.
+                                writeInt((bodyEnd - objInfo.position()) + footerLen);
+                            else
+                                writeInt(objInfo.length());
+                        }
+                    }
+                }
+                else
+                    for (int fieldLen : data)
+                        // writing field len and resetting is handle mask
+                        writeInt(fieldLen & ~FOOTER_BODY_IS_HANDLE_MASK);
+
+                writeShort(footerLen);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamExtRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamExtRegistry.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamExtRegistry.java
new file mode 100644
index 0000000..f26bacb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectStreamExtRegistry.java
@@ -0,0 +1,225 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.io.*;
+import java.util.concurrent.*;
+
+/**
+ * Storage for object streams.
+ */
+class OptimizedObjectStreamExtRegistry {
+    /** Holders. */
+    private static final ThreadLocal<StreamHolder> holders = new ThreadLocal<>();
+
+    /** Holders pool. */
+    private static BlockingQueue<StreamHolder> pool;
+
+    /**
+     * Ensures singleton.
+     */
+    private OptimizedObjectStreamExtRegistry() {
+        // No-op.
+    }
+
+    /**
+     * Sets streams pool size.
+     *
+     * @param size Streams pool size.
+     */
+    static void poolSize(int size) {
+        if (size > 0) {
+            pool = new LinkedBlockingQueue<>(size);
+
+            for (int i = 0; i < size; i++) {
+                boolean b = pool.offer(new StreamHolder());
+
+                assert b;
+            }
+        }
+        else
+            pool = null;
+    }
+
+    /**
+     * Gets output stream.
+     *
+     * @return Object output stream.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    static OptimizedObjectOutputStreamExt out() throws IgniteInterruptedCheckedException {
+        return holder().acquireOut();
+    }
+
+    /**
+     * Gets input stream.
+     *
+     * @return Object input stream.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    static OptimizedObjectInputStreamExt in() throws IgniteInterruptedCheckedException {
+        return holder().acquireIn();
+    }
+
+    /**
+     * Closes and releases output stream.
+     *
+     * @param out Object output stream.
+     */
+    static void closeOut(OptimizedObjectOutputStream out) {
+        U.close(out, null);
+
+        StreamHolder holder = holders.get();
+
+        holder.releaseOut();
+
+        if (pool != null) {
+            holders.remove();
+
+            boolean b = pool.offer(holder);
+
+            assert b;
+        }
+    }
+
+    /**
+     * Closes and releases input stream.
+     *
+     * @param in Object input stream.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    static void closeIn(OptimizedObjectInputStream in) {
+        U.close(in, null);
+
+        StreamHolder holder = holders.get();
+
+        holder.releaseIn();
+
+        if (pool != null) {
+            holders.remove();
+
+            boolean b = pool.offer(holder);
+
+            assert b;
+        }
+    }
+
+    /**
+     * Gets holder from pool or thread local.
+     *
+     * @return Stream holder.
+     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
+     */
+    private static StreamHolder holder() throws IgniteInterruptedCheckedException {
+        StreamHolder holder = holders.get();
+
+        if (holder == null) {
+            try {
+                holders.set(holder = pool != null ? pool.take() : new StreamHolder());
+            }
+            catch (InterruptedException e) {
+                throw new IgniteInterruptedCheckedException(
+                    "Failed to take object stream from pool (thread interrupted).", e);
+            }
+        }
+
+        return holder;
+    }
+
+    /**
+     * Streams holder.
+     */
+    private static class StreamHolder {
+        /** Output stream. */
+        private final OptimizedObjectOutputStreamExt out = createOut();
+
+        /** Input stream. */
+        private final OptimizedObjectInputStreamExt in = createIn();
+
+        /** Output streams counter. */
+        private int outAcquireCnt;
+
+        /** Input streams counter. */
+        private int inAcquireCnt;
+
+        /**
+         * Gets output stream.
+         *
+         * @return Object output stream.
+         */
+        OptimizedObjectOutputStreamExt acquireOut() {
+            return outAcquireCnt++ > 0 ? createOut() : out;
+        }
+
+        /**
+         * Gets input stream.
+         *
+         * @return Object input stream.
+         */
+        OptimizedObjectInputStreamExt acquireIn() {
+            return inAcquireCnt++ > 0 ? createIn() : in;
+        }
+
+        /**
+         * Releases output stream.
+         */
+        void releaseOut() {
+            outAcquireCnt--;
+        }
+
+        /**
+         * Releases input stream.
+         */
+        void releaseIn() {
+            inAcquireCnt--;
+        }
+
+        /**
+         * Creates output stream.
+         *
+         * @return Object output stream.
+         */
+        private OptimizedObjectOutputStreamExt createOut() {
+            try {
+                return new OptimizedObjectOutputStreamExt(new GridUnsafeDataOutput(4 * 1024));
+            }
+            catch (IOException e) {
+                throw new IgniteException("Failed to create object output stream.", e);
+            }
+        }
+
+        /**
+         * Creates input stream.
+         *
+         * @return Object input stream.
+         */
+        private OptimizedObjectInputStreamExt createIn() {
+            try {
+                return new OptimizedObjectInputStreamExt(new GridUnsafeDataInput());
+            }
+            catch (IOException e) {
+                throw new IgniteException("Failed to create object input stream.", e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
deleted file mode 100644
index 6bfe9c6..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
+++ /dev/null
@@ -1,353 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-
-import static org.apache.ignite.marshaller.optimized.OptimizedClassDescriptor.*;
-import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
-
-/**
- * TODO
- */
-public class OptimizedMarshallerExt extends OptimizedMarshaller {
-    /** */
-    static final byte EMPTY_FOOTER = -1;
-
-    /** */
-    static final byte FOOTER_LEN_OFF = 2;
-
-    /** */
-    static final int FOOTER_BODY_LEN_MASK = 0x3FFFFFFF;
-
-    /** */
-    static final int FOOTER_BODY_IS_HANDLE_MASK = 0x40000000;
-
-    /** */
-    static final byte FOOTER_BODY_HANDLE_MASK_BIT = 30;
-
-    /** */
-    static final byte VARIABLE_LEN = -1;
-
-    /** */
-    private volatile OptimizedMarshallerExtMetaHandler metaHandler;
-
-    /**
-     * Creates new marshaller will all defaults.
-     *
-     * @throws IgniteException If this marshaller is not supported on the current JVM.
-     */
-    public OptimizedMarshallerExt() {
-        // No-op
-    }
-
-    /**
-     * Creates new marshaller providing whether it should
-     * require {@link Serializable} interface or not.
-     *
-     * @param requireSer Whether to require {@link Serializable}.
-     */
-    public OptimizedMarshallerExt(boolean requireSer) {
-        super(requireSer);
-    }
-
-    /**
-     * Sets metadata handler.
-     *
-     * @param metaHandler Metadata handler.
-     */
-    public void setMetadataHandler(OptimizedMarshallerExtMetaHandler metaHandler) {
-        this.metaHandler = metaHandler;
-    }
-
-    /**
-     * Returns currently set ID mapper.
-     *
-     * @return ID mapper.
-     */
-    public OptimizedMarshallerIdMapper idMapper() {
-        return mapper;
-    }
-
-    /**
-     * Enables fields indexing for the object of the given {@code cls}.
-     *
-     * If enabled then a footer will be added during marshalling of an object of the given {@code cls} to the end of
-     * its serialized form.
-     *
-     * @param cls Class.
-     * @return {@code true} if fields indexing is enabled.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public boolean enableFieldsIndexing(Class<?> cls) throws IgniteCheckedException {
-        assert metaHandler != null;
-
-        if (ctx.isSystemType(cls.getName()))
-            return false;
-
-        try {
-            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
-
-            if (desc.fields() != null && desc.fields().fieldsIndexingSupported()) {
-                //The function is called on kernel startup, calling metaHandler.metadata() will hang the grid,
-                //because the underlying cache is not ready.
-                //if (metaHandler.metadata(desc.typeId()) != null)
-                //    return true;
-
-                OptimizedObjectMetadata meta = new OptimizedObjectMetadata();
-
-                for (ClassFields clsFields : desc.fields().fieldsList())
-                    for (FieldInfo info : clsFields.fieldInfoList())
-                        meta.addMeta(info.id(), info.type());
-
-                metaHandler.addMeta(desc.typeId(), meta);
-
-                return true;
-            }
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to put meta for class: " + cls.getName(), e);
-        }
-
-        return false;
-    }
-
-    /**
-     * Checks whether fields indexing is enabled for objects of the given {@code cls}.
-     *
-     * @param cls Class.
-     * @return {@code true} if fields indexing is enabled.
-     */
-    public boolean fieldsIndexingEnabled(Class<?> cls) {
-        assert metaHandler != null;
-
-        if (ctx.isSystemType(cls.getName()))
-            return false;
-
-        try {
-            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
-
-            return desc.fields() != null && desc.fields().fieldsIndexingSupported() &&
-                metaHandler.metadata(desc.typeId()) != null;
-        }
-        catch (IOException e) {
-            throw new IgniteException("Failed to load class description: " + cls);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setPoolSize(int poolSize) {
-        OptimizedObjectStreamExtRegistry.poolSize(poolSize);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
-        assert out != null;
-
-        OptimizedObjectOutputStreamExt objOut = null;
-
-        try {
-            objOut = OptimizedObjectStreamExtRegistry.out();
-
-            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
-
-            objOut.out().outputStream(out);
-
-            objOut.writeObject(obj);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeOut(objOut);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
-        OptimizedObjectOutputStreamExt objOut = null;
-
-        try {
-            objOut = OptimizedObjectStreamExtRegistry.out();
-
-            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
-
-            objOut.writeObject(obj);
-
-            return objOut.out().array();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeOut(objOut);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        assert in != null;
-
-        OptimizedObjectInputStreamExt objIn = null;
-
-        try {
-            objIn = OptimizedObjectStreamExtRegistry.in();
-
-            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
-
-            objIn.in().inputStream(in);
-
-            return (T)objIn.readObject();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
-        }
-        catch (ClassNotFoundException e) {
-            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                                             "(make sure same versions of all classes are available on all nodes or " +
-                                             "enable peer-class-loading): " + clsLdr, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeIn(objIn);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public <T> T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        return unmarshal(arr, 0, arr.length, clsLdr);
-    }
-
-    /**
-     * Unmarshals object from byte array using given class loader and offset with len.
-     *
-     * @param <T> Type of unmarshalled object.
-     * @param arr Byte array.
-     * @param off Object's offset in the array.
-     * @param len Object's length in the array.
-     * @param clsLdr Class loader to use.
-     * @return Unmarshalled object.
-     * @throws IgniteCheckedException If unmarshalling failed.
-     */
-     public <T> T unmarshal(byte[] arr, int off, int len, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
-        assert arr != null;
-
-        OptimizedObjectInputStreamExt objIn = null;
-
-        try {
-            objIn = OptimizedObjectStreamExtRegistry.in();
-
-            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
-
-            objIn.in().bytes(arr, off, len);
-
-            return (T)objIn.readObject();
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
-        }
-        catch (ClassNotFoundException e) {
-            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                                                 "(make sure same version of all classes are available on all nodes or" +
-                                                 " enable peer-class-loading): " + clsLdr, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeIn(objIn);
-        }
-    }
-
-    /**
-     * Checks whether object, serialized to byte array {@code arr}, has a field with name {@code fieldName}.
-     *
-     * @param fieldName Field name.
-     * @param arr Object's serialized form.
-     * @param off Object's start off.
-     * @param len Object's len.
-     * @return {@code true} if field exists.
-     */
-    public boolean hasField(String fieldName, byte[] arr, int off, int len) throws IgniteCheckedException {
-        assert arr != null && fieldName != null;
-
-        OptimizedObjectInputStreamExt objIn = null;
-
-        try {
-            objIn = OptimizedObjectStreamExtRegistry.in();
-
-            objIn.context(clsMap, ctx, mapper, dfltClsLdr, metaHandler);
-
-            objIn.in().bytes(arr, off, len);
-
-            return objIn.hasField(fieldName);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeIn(objIn);
-        }
-    }
-
-    /**
-     * Looks up field with the given name and returns it in one of the following representations. If the field is
-     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
-     * for future processing. In all other cases the field is fully deserialized.
-     *
-     * @param fieldName Field name.
-     * @param arr Object's serialized form.
-     * @param off Object's start offset.
-     * @param len Object's len.
-     * @param clsLdr Class loader.
-     * @param <T> Expected field class.
-     * @return Field.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public <T> T readField(String fieldName, byte[] arr, int off, int len, @Nullable ClassLoader clsLdr)
-        throws IgniteCheckedException {
-
-        assert arr != null && fieldName != null;
-
-        OptimizedObjectInputStreamExt objIn = null;
-
-        try {
-            objIn = OptimizedObjectStreamExtRegistry.in();
-
-            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
-
-            objIn.in().bytes(arr, off, len);
-
-            return objIn.readField(fieldName);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
-        }
-        catch (ClassNotFoundException e) {
-            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                                             "(make sure same version of all classes are available on all nodes or" +
-                                             " enable peer-class-loading): " + clsLdr, e);
-        }
-        finally {
-            OptimizedObjectStreamExtRegistry.closeIn(objIn);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtMetaHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtMetaHandler.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtMetaHandler.java
deleted file mode 100644
index ea3b70f..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtMetaHandler.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-/**
- * Metadata handler for optimized objects.
- */
-public interface OptimizedMarshallerExtMetaHandler {
-    /**
-     * Adds meta data.
-     *
-     * @param typeId Type ID.
-     * @param meta Meta data.
-     */
-    void addMeta(int typeId, OptimizedObjectMetadata meta);
-
-
-    /**
-     * Gets meta data for provided type ID.
-     *
-     * @param typeId Type ID.
-     * @return Meta data.
-     */
-    OptimizedObjectMetadata metadata(int typeId);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
deleted file mode 100644
index 66544a0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
+++ /dev/null
@@ -1,241 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.optimized.*;
-
-import java.io.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
-import static org.apache.ignite.marshaller.optimized.ext.OptimizedMarshallerExt.*;
-
-
-/**
- * TODO: IGNITE-950
- */
-public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
-    /** */
-    private OptimizedMarshallerExtMetaHandler metaHandler;
-
-    /** {@inheritDoc} */
-    public OptimizedObjectInputStreamExt(GridDataInput in) throws IOException {
-        super(in);
-    }
-
-    /**
-     * @param clsMap Class descriptors by class map.
-     * @param ctx Context.
-     * @param mapper ID mapper.
-     * @param clsLdr Class loader.
-     */
-    protected void context(ConcurrentMap<Class, OptimizedClassDescriptor> clsMap, MarshallerContext ctx,
-        OptimizedMarshallerIdMapper mapper, ClassLoader clsLdr, OptimizedMarshallerExtMetaHandler metaHandler) {
-        context(clsMap, ctx, mapper, clsLdr);
-
-        this.metaHandler = metaHandler;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void skipFooter(Class<?> cls) throws IOException {
-        if (!ctx.isSystemType(cls.getName()) && metaHandler != null &&
-            metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null) {
-            short footerLen = in.readShort();
-
-            if (footerLen != EMPTY_FOOTER)
-                in.skipBytes(footerLen - 2);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected int readFieldType() throws IOException {
-        return in.readByte();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        super.close();
-
-        metaHandler = null;
-    }
-
-    /**
-     * Checks whether the object has a field with name {@code fieldName}.
-     *
-     * @param fieldName Field name.
-     * @return {@code true} if field exists, {@code false} otherwise.
-     * @throws IOException in case of error.
-     */
-    boolean hasField(String fieldName) throws IOException {
-        int pos = in.position();
-
-        if (in.readByte() != SERIALIZABLE) {
-            in.position(pos);
-            return false;
-        }
-
-        FieldRange range = fieldRange(fieldName, pos);
-
-        in.position(pos);
-
-        return range != null && range.start > 0;
-    }
-
-    /**
-     * Looks up field with the given name and returns it in one of the following representations. If the field is
-     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
-     * for future processing. In all other cases the field is fully deserialized.
-     *
-     * @param fieldName Field name.
-     * @return Field.
-     * @throws IOException In case of error.
-     * @throws ClassNotFoundException In case of error.
-     */
-    <F> F readField(String fieldName) throws IOException, ClassNotFoundException {
-        int pos = in.position();
-
-        if (in.readByte() != SERIALIZABLE) {
-            in.position(pos);
-            return null;
-        }
-
-        FieldRange range = fieldRange(fieldName, pos);
-
-        F field = null;
-
-        if (range != null && range.start >= 0) {
-            in.position(range.start);
-
-            if (in.readByte() == SERIALIZABLE && metaHandler.metadata(in.readInt()) != null)
-                //Do we need to make a copy of array?
-                field = (F)new CacheIndexedObjectImpl(in.array(), range.start, range.len);
-            else {
-                in.position(range.start);
-                field = (F)readObject();
-            }
-        }
-
-        in.position(pos);
-
-        return field;
-    }
-
-    /**
-     * Returns field offset in the byte stream.
-     *
-     * @param fieldName Field name.
-     * @param start Object's start offset.
-     * @return positive range or {@code null} if the object doesn't have such a field.
-     * @throws IOException in case of error.
-     */
-    private FieldRange fieldRange(String fieldName, int start) throws IOException {
-        int fieldId = resolveFieldId(fieldName);
-
-        int typeId = readInt();
-
-        int clsNameLen = 0;
-
-        if (typeId == 0) {
-            int pos = in.position();
-
-            typeId = OptimizedMarshallerUtils.resolveTypeId(readUTF(), mapper);
-
-            clsNameLen = in.position() - pos;
-        }
-
-        OptimizedObjectMetadata meta = metaHandler.metadata(typeId);
-
-        if (meta == null)
-            // TODO: IGNITE-950 add warning!
-            return null;
-
-        int end = in.size();
-
-        in.position(end - FOOTER_LEN_OFF);
-
-        short footerLen = in.readShort();
-
-        if (footerLen == EMPTY_FOOTER)
-            return null;
-
-        // +2 - skipping length at the beginning
-        int footerOff = (end - footerLen) + 2;
-        in.position(footerOff);
-
-        int fieldOff = 0;
-
-        for (OptimizedObjectMetadata.FieldInfo info : meta.getMeta()) {
-            int len;
-            boolean isHandle;
-
-            if (info.len == VARIABLE_LEN) {
-                int fieldInfo = in.readInt();
-
-                len = fieldInfo & FOOTER_BODY_LEN_MASK;
-                isHandle = ((fieldInfo & FOOTER_BODY_IS_HANDLE_MASK) >> FOOTER_BODY_HANDLE_MASK_BIT) == 1;
-            }
-            else {
-                len = info.len;
-                isHandle = false;
-            }
-
-            if (info.id == fieldId) {
-                if (!isHandle) {
-                    //object header len: 1 - for type, 4 - for type ID, 2 - for checksum.
-                    fieldOff += 1 + 4 + clsNameLen + 2;
-
-                    return new FieldRange(start + fieldOff, len);
-                }
-                else
-                    return new FieldRange(in.readInt(), in.readInt());
-            }
-            else {
-                fieldOff += len;
-
-                if (isHandle) {
-                    in.skipBytes(8);
-                    fieldOff += 8;
-                }
-            }
-        }
-
-        return null;
-    }
-
-    /**
-     *
-     */
-    private static class FieldRange {
-        /** */
-        private int start;
-
-        /** */
-        private int len;
-
-        /**
-         * @param start Start.
-         * @param len   Length.
-         */
-        public FieldRange(int start, int len) {
-            this.start = start;
-            this.len = len;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
deleted file mode 100644
index b3b2ecc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadata.java
+++ /dev/null
@@ -1,157 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.*;
-import org.apache.ignite.marshaller.optimized.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- * Metadata that keeps fields information. Used in conjunction with the footer that is added to some objects during
- * marshalling.
- */
-public class OptimizedObjectMetadata implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private List<FieldInfo> fieldsInfo;
-
-    /** Constructor. */
-    public OptimizedObjectMetadata() {
-        // No-op
-    }
-
-    /**
-     * Adds meta for a new field.
-     *
-     * @param fieldId Field ID.
-     * @param fieldType Field type.
-     */
-    public void addMeta(int fieldId, OptimizedFieldType fieldType) {
-        if (fieldsInfo == null)
-            fieldsInfo = new ArrayList<>();
-
-
-
-        fieldsInfo.add(new FieldInfo(fieldId, fieldType));
-    }
-
-    /**
-     * Gets {@link OptimizedObjectMetadata.FieldInfo} at the {@code index}.
-     *
-     * @param index Position.
-     * @return Field meta info.
-     */
-    public FieldInfo getMeta(int index) {
-        return fieldsInfo.get(index);
-    }
-    /**
-     * Returns all the metadata stored for the object.
-     *
-     * @return Metadata collection.
-     */
-    public List<FieldInfo> getMeta() {
-        return Collections.unmodifiableList(fieldsInfo);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        if (fieldsInfo == null) {
-            out.writeInt(0);
-            return;
-        }
-
-        out.writeInt(fieldsInfo.size());
-
-        for (FieldInfo fieldInfo : fieldsInfo) {
-            out.writeInt(fieldInfo.id);
-            out.writeByte(fieldInfo.type.ordinal());
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        int size = in.readInt();
-
-        fieldsInfo = new ArrayList<>(size);
-
-        for (int i = 0; i < size; i++)
-            fieldsInfo.add(new FieldInfo(in.readInt(), OptimizedFieldType.values()[in.readByte()]));
-    }
-
-    /**
-     * Field info.
-     */
-    public static class FieldInfo {
-        /** Field ID. */
-        int id;
-
-        /** Field len. */
-        int len;
-
-        /** Field type. */
-        OptimizedFieldType type;
-
-        /**
-         * Constructor.
-         *
-         * @param id Field ID.
-         * @param type Field len.
-         */
-        public FieldInfo(int id, OptimizedFieldType type) {
-            this.id = id;
-            this.type = type;
-
-            len = 1;
-
-            switch (type) {
-                case BYTE:
-                case BOOLEAN:
-                    len += 1;
-                    break;
-
-                case SHORT:
-                case CHAR:
-                    len += 2;
-                    break;
-
-                case INT:
-                case FLOAT:
-                    len += 4;
-                    break;
-
-                case LONG:
-                case DOUBLE:
-                    len += 8;
-                    break;
-
-                case OTHER:
-                    len = OptimizedMarshallerExt.VARIABLE_LEN;
-                    break;
-
-                default:
-                    throw new IgniteException("Unknown field type: " + type);
-            }
-
-            assert len != 1;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
deleted file mode 100644
index b9fcd58..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectMetadataKey.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.internal.processors.cache.*;
-
-import java.io.*;
-
-/**
- * Optimized object metadata key.
- */
-public class OptimizedObjectMetadataKey extends GridCacheUtilityKey<OptimizedObjectMetadataKey>
-    implements Externalizable {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private int typeId;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public OptimizedObjectMetadataKey() {
-        // No-op
-    }
-
-    /**
-     * Constructor.
-     *
-     * @param typeId Type id.
-     */
-    public OptimizedObjectMetadataKey(int typeId) {
-        this.typeId = typeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        out.writeInt(typeId);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        typeId = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean equalsx(OptimizedObjectMetadataKey key) {
-        return typeId == key.typeId;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        return typeId;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
deleted file mode 100644
index 8ae8f9e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.optimized.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static org.apache.ignite.marshaller.optimized.ext.OptimizedMarshallerExt.*;
-import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
-
-/**
- * TODO: IGNITE-950
- */
-public class OptimizedObjectOutputStreamExt extends OptimizedObjectOutputStream {
-    /** */
-    private OptimizedMarshallerExtMetaHandler metaHandler;
-
-    /**
-     * Constructor.
-     *
-     * @param out Output stream.
-     * @throws IOException In case of error.
-     */
-    protected OptimizedObjectOutputStreamExt(GridDataOutput out) throws IOException {
-        super(out);
-    }
-
-    /**
-     * @param clsMap Class descriptors by class map.
-     * @param ctx Context.
-     * @param mapper ID mapper.
-     * @param requireSer Require {@link Serializable} flag.
-     * @param metaHandler Metadata handler.
-     */
-    protected void context(ConcurrentMap<Class, OptimizedClassDescriptor> clsMap, MarshallerContext ctx,
-        OptimizedMarshallerIdMapper mapper, boolean requireSer, OptimizedMarshallerExtMetaHandler metaHandler) {
-        context(clsMap, ctx, mapper, requireSer);
-
-        this.metaHandler = metaHandler;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void writeFieldType(byte type) throws IOException {
-        out.writeByte(type);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Footer createFooter(Class<?> cls) {
-        if (!ctx.isSystemType(cls.getName()) && metaHandler != null &&
-            metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null)
-            return new FooterImpl();
-        else
-            return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() throws IOException {
-        super.close();
-
-        metaHandler = null;
-    }
-
-    /**
-     *
-     */
-    private class FooterImpl implements OptimizedObjectOutputStream.Footer {
-        /** */
-        private ArrayList<Integer> data;
-
-        /** */
-        private ArrayList<Integer> fields;
-
-        /** */
-        private HashMap<Integer, GridHandleTable.ObjectInfo> handles;
-
-        /** */
-        private boolean hasHandles;
-
-        /** {@inheritDoc} */
-        @Override public void fields(OptimizedClassDescriptor.Fields fields) {
-            if (fields.fieldsIndexingSupported()) {
-                data = new ArrayList<>();
-                this.fields = new ArrayList<>();
-            }
-            else
-                data = null;
-        }
-
-        /** {@inheritDoc} */
-        public void put(int fieldId, OptimizedFieldType fieldType, int len) {
-            if (data == null)
-                return;
-
-            if (fieldType == OptimizedFieldType.OTHER) {
-                data.add(len);
-                fields.add(fieldId);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void putHandle(int fieldId, GridHandleTable.ObjectInfo objInfo) {
-            if (data == null)
-                return;
-
-            if (!hasHandles) {
-                hasHandles = true;
-                handles = new HashMap<>();
-            }
-
-            handles.put(fieldId, objInfo);
-
-            // length of handle fields is 5 bytes.
-            put(fieldId, OptimizedFieldType.OTHER, 5);
-        }
-
-        /** {@inheritDoc} */
-        public void write() throws IOException {
-            if (data == null)
-                writeInt(EMPTY_FOOTER);
-            else {
-                int bodyEnd = out.offset();
-
-                // +4 - 2 bytes for footer len at the beginning, 2 bytes for footer len at the end.
-                short footerLen = (short)(data.size() * 4 + 4);
-
-                if (hasHandles)
-                    footerLen += handles.size() * 8;
-
-                writeShort(footerLen);
-
-                if (hasHandles) {
-                    for (int i = 0; i < data.size(); i++) {
-                        GridHandleTable.ObjectInfo objInfo = handles.get(fields.get(i));
-
-                        if (objInfo == null)
-                            writeInt(data.get(i) & ~FOOTER_BODY_IS_HANDLE_MASK);
-                        else {
-                            writeInt(data.get(i) | FOOTER_BODY_IS_HANDLE_MASK);
-                            writeInt(objInfo.position());
-
-                            if (objInfo.length() == 0)
-                                // field refers to its own object that hasn't set total length yet.
-                                writeInt((bodyEnd - objInfo.position()) + footerLen);
-                            else
-                                writeInt(objInfo.length());
-                        }
-                    }
-                }
-                else
-                    for (int fieldLen : data)
-                        // writing field len and resetting is handle mask
-                        writeInt(fieldLen & ~FOOTER_BODY_IS_HANDLE_MASK);
-
-                writeShort(footerLen);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
deleted file mode 100644
index e07b4de..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectStreamExtRegistry.java
+++ /dev/null
@@ -1,226 +0,0 @@
-/*
- * 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.marshaller.optimized.ext;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.*;
-
-import java.io.*;
-import java.util.concurrent.*;
-
-/**
- * Storage for object streams.
- */
-class OptimizedObjectStreamExtRegistry {
-    /** Holders. */
-    private static final ThreadLocal<StreamHolder> holders = new ThreadLocal<>();
-
-    /** Holders pool. */
-    private static BlockingQueue<StreamHolder> pool;
-
-    /**
-     * Ensures singleton.
-     */
-    private OptimizedObjectStreamExtRegistry() {
-        // No-op.
-    }
-
-    /**
-     * Sets streams pool size.
-     *
-     * @param size Streams pool size.
-     */
-    static void poolSize(int size) {
-        if (size > 0) {
-            pool = new LinkedBlockingQueue<>(size);
-
-            for (int i = 0; i < size; i++) {
-                boolean b = pool.offer(new StreamHolder());
-
-                assert b;
-            }
-        }
-        else
-            pool = null;
-    }
-
-    /**
-     * Gets output stream.
-     *
-     * @return Object output stream.
-     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
-     */
-    static OptimizedObjectOutputStreamExt out() throws IgniteInterruptedCheckedException {
-        return holder().acquireOut();
-    }
-
-    /**
-     * Gets input stream.
-     *
-     * @return Object input stream.
-     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
-     */
-    static OptimizedObjectInputStreamExt in() throws IgniteInterruptedCheckedException {
-        return holder().acquireIn();
-    }
-
-    /**
-     * Closes and releases output stream.
-     *
-     * @param out Object output stream.
-     */
-    static void closeOut(OptimizedObjectOutputStream out) {
-        U.close(out, null);
-
-        StreamHolder holder = holders.get();
-
-        holder.releaseOut();
-
-        if (pool != null) {
-            holders.remove();
-
-            boolean b = pool.offer(holder);
-
-            assert b;
-        }
-    }
-
-    /**
-     * Closes and releases input stream.
-     *
-     * @param in Object input stream.
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    static void closeIn(OptimizedObjectInputStream in) {
-        U.close(in, null);
-
-        StreamHolder holder = holders.get();
-
-        holder.releaseIn();
-
-        if (pool != null) {
-            holders.remove();
-
-            boolean b = pool.offer(holder);
-
-            assert b;
-        }
-    }
-
-    /**
-     * Gets holder from pool or thread local.
-     *
-     * @return Stream holder.
-     * @throws IgniteInterruptedCheckedException If thread is interrupted while trying to take holder from pool.
-     */
-    private static StreamHolder holder() throws IgniteInterruptedCheckedException {
-        StreamHolder holder = holders.get();
-
-        if (holder == null) {
-            try {
-                holders.set(holder = pool != null ? pool.take() : new StreamHolder());
-            }
-            catch (InterruptedException e) {
-                throw new IgniteInterruptedCheckedException(
-                    "Failed to take object stream from pool (thread interrupted).", e);
-            }
-        }
-
-        return holder;
-    }
-
-    /**
-     * Streams holder.
-     */
-    private static class StreamHolder {
-        /** Output stream. */
-        private final OptimizedObjectOutputStreamExt out = createOut();
-
-        /** Input stream. */
-        private final OptimizedObjectInputStreamExt in = createIn();
-
-        /** Output streams counter. */
-        private int outAcquireCnt;
-
-        /** Input streams counter. */
-        private int inAcquireCnt;
-
-        /**
-         * Gets output stream.
-         *
-         * @return Object output stream.
-         */
-        OptimizedObjectOutputStreamExt acquireOut() {
-            return outAcquireCnt++ > 0 ? createOut() : out;
-        }
-
-        /**
-         * Gets input stream.
-         *
-         * @return Object input stream.
-         */
-        OptimizedObjectInputStreamExt acquireIn() {
-            return inAcquireCnt++ > 0 ? createIn() : in;
-        }
-
-        /**
-         * Releases output stream.
-         */
-        void releaseOut() {
-            outAcquireCnt--;
-        }
-
-        /**
-         * Releases input stream.
-         */
-        void releaseIn() {
-            inAcquireCnt--;
-        }
-
-        /**
-         * Creates output stream.
-         *
-         * @return Object output stream.
-         */
-        private OptimizedObjectOutputStreamExt createOut() {
-            try {
-                return new OptimizedObjectOutputStreamExt(new GridUnsafeDataOutput(4 * 1024));
-            }
-            catch (IOException e) {
-                throw new IgniteException("Failed to create object output stream.", e);
-            }
-        }
-
-        /**
-         * Creates input stream.
-         *
-         * @return Object input stream.
-         */
-        private OptimizedObjectInputStreamExt createIn() {
-            try {
-                return new OptimizedObjectInputStreamExt(new GridUnsafeDataInput());
-            }
-            catch (IOException e) {
-                throw new IgniteException("Failed to create object input stream.", e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
deleted file mode 100644
index 84d1ce3..0000000
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/package-info.java
+++ /dev/null
@@ -1,21 +0,0 @@
-/*
- * 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 description. -->
- * Contains <b>extended</b> version of Optimized marshaller.
- */
-package org.apache.ignite.marshaller.optimized.ext;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
index c1f07ce..7866500 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import java.io.*;
 import java.util.concurrent.*;
 
 /**
@@ -33,7 +34,7 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
     private static ConcurrentHashMap<Integer, OptimizedObjectMetadata> META_BUF = new ConcurrentHashMap<>();
 
     /** */
-    private static final OptimizedMarshallerExtMetaHandler META_HANDLER = new OptimizedMarshallerExtMetaHandler() {
+    private static final OptimizedMarshallerMetaHandler META_HANDLER = new OptimizedMarshallerMetaHandler() {
         @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
             META_BUF.putIfAbsent(typeId, meta);
         }
@@ -143,6 +144,29 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
         assertEquals(selfLinkObject, selfLinkObject2);
     }
 
+
+    /**
+     * @throws Exception In case of error.
+     */
+    /*public void testMarshalAware() throws Exception {
+        META_BUF.clear();
+
+        OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
+
+        assertTrue(marsh.enableFieldsIndexing(TestMarshalAware.class));
+        assertEquals(0, META_BUF.size());
+
+        TestMarshalAware test = new TestMarshalAware(100, "MarshalAware");
+
+        byte[] arr = marsh.marshal(test);
+
+        assertEquals(1, META_BUF.size());
+
+        TestMarshalAware test2 = marsh.unmarshal(arr, null);
+
+        assertEquals(test, test2);
+    }*/
+
     private static class InternalMarshaller extends OptimizedMarshallerExt {
         /**
          * Constructor.
@@ -161,7 +185,7 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
         }
 
         /** {@inheritDoc} */
-        @Override public void setMetadataHandler(OptimizedMarshallerExtMetaHandler metaHandler) {
+        @Override public void setMetadataHandler(OptimizedMarshallerMetaHandler metaHandler) {
             // No-op
         }
     }
@@ -281,4 +305,51 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
             return true;
         }
     }
+
+    /**
+     *
+     */
+    private static class TestMarshalAware implements OptimizedMarshalAware {
+        /** */
+        private int i;
+
+        /** */
+        private String str;
+
+        public TestMarshalAware() {
+            // No-op
+        }
+
+        public TestMarshalAware(int i, String str) {
+            this.i = i;
+            this.str = str;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeFields(OptimizedFieldsWriter writer) throws IOException {
+            writer.writeInt("i", i);
+            writer.writeString("str", str);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readFields(OptimizedFieldsReader reader) throws IOException {
+            i = reader.readInt("i");
+            str = reader.readString("str");
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o)
+                return true;
+            if (o == null || getClass() != o.getClass())
+                return false;
+
+            TestMarshalAware that = (TestMarshalAware)o;
+
+            if (i != that.i)
+                return false;
+
+            return !(str != null ? !str.equals(that.str) : that.str != null);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
index e7609d6..929bdf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteTestResources.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.logger.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOptimizedMarshallerExtQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOptimizedMarshallerExtQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOptimizedMarshallerExtQuerySelfTest.java
index 010f3e6..77e5bb2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOptimizedMarshallerExtQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOptimizedMarshallerExtQuerySelfTest.java
@@ -21,9 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.query.*;
-import org.apache.ignite.internal.processors.query.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.marshaller.optimized.*;
 
 import javax.cache.*;
 import java.util.*;


[24/33] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: a4ce455bbff7dbbf1084a7e9cc60424b53396471
Parents: 7c70267 6f5e676
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 22 12:50:21 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 22 12:50:21 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 45 ++++++++++----------
 1 file changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[04/33] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1034' into ignite-1034

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-1034' into ignite-1034


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

Branch: refs/heads/ignite-950
Commit: ef4abeba881f079cab08af9ade99fe52ecf5d26d
Parents: 462495f 49dc4a5
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 12:02:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 12:02:31 2015 +0300

----------------------------------------------------------------------

----------------------------------------------------------------------



[14/33] incubator-ignite git commit: #IGNITE-1034 - Corrected messages after code review.

Posted by sb...@apache.org.
#IGNITE-1034 - Corrected messages after code review.


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

Branch: refs/heads/ignite-950
Commit: 03dc03d42eeabedbb82862461b523421e49744b9
Parents: e1b8800
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 11:32:23 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 11:32:23 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03dc03d4/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 3997768..1ec2128 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1917,7 +1917,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                 if (node != null && node.isClient()) {
                     String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
-                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [" +
+                        "srvNode=" + getSpiContext().localNode().id() +
+                        ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
                     LT.warn(log, null, msg);


[09/33] incubator-ignite git commit: ignite-950: putting meta to local cache

Posted by sb...@apache.org.
ignite-950: putting meta to local cache


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

Branch: refs/heads/ignite-950
Commit: 9fb9679d0e236b6fbb8a386fabb8907d79707e0e
Parents: 5fe19ff
Author: Denis Magda <dm...@gridgain.com>
Authored: Fri Jun 19 17:10:28 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Fri Jun 19 17:10:28 2015 +0300

----------------------------------------------------------------------
 .../processors/cacheobject/IgniteCacheObjectProcessorImpl.java     | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9fb9679d/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 64aa064..748dd42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -115,6 +115,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                     if (meta == null)
                         meta = EMPTY_META;
 
+                    metaBuf.put(typeId, meta);
+
                     return meta == EMPTY_META ? null : meta;
                 }
             };


[23/33] incubator-ignite git commit: # review droppng of slow clients

Posted by sb...@apache.org.
# review droppng of slow clients


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

Branch: refs/heads/ignite-950
Commit: 7c70267d2b4b7fc3fb7a1efda1ee785cebebcc41
Parents: 1cc0000
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 22 12:49:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 22 12:49:58 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  3 ++
 .../communication/tcp/TcpCommunicationSpi.java  | 49 ++++++++++----------
 .../tcp/TcpCommunicationSpiMBean.java           |  8 ++++
 3 files changed, 36 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index b166f39..542fa30 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -236,7 +236,10 @@ public final class IgniteSystemProperties {
     /**
      * System property to disable buffered communication if node sends less messages count than
      * specified by this property. Default value is {@code 512}.
+     *
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     public static final String IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT = "IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT";
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1ec2128..1035ee5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.ipc.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
-import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
@@ -51,7 +50,6 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.IgniteSystemProperties.*;
 import static org.apache.ignite.events.EventType.*;
 
 /**
@@ -657,9 +655,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Slow client queue limit. */
     private int slowClientQueueLimit;
 
-    /** Min buffered message count. */
-    private int minBufferedMsgCnt = Integer.getInteger(IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT, 512);
-
     /** NIO server. */
     private GridNioServer<Message> nioSrvr;
 
@@ -935,24 +930,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      *
      * @param connBufSize Connection buffer size.
      * @see #setConnectionBufferFlushFrequency(long)
+     * @deprecated Not used any more.
      */
+    @Deprecated
     @IgniteSpiConfiguration(optional = true)
     public void setConnectionBufferSize(int connBufSize) {
         // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public int getConnectionBufferSize() {
         return 0;
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @IgniteSpiConfiguration(optional = true)
     @Override public void setConnectionBufferFlushFrequency(long connBufFlushFreq) {
         // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public long getConnectionBufferFlushFrequency() {
         return 0;
     }
@@ -1156,13 +1156,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * Sets slow client queue limit.
      * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * When set to a positive number, communication SPI will monitor clients outbound message queue sizes and will drop
      * those clients whose queue exceeded this limit.
      * <p/>
      * Usually this value should be set to the same value as {@link #getMessageQueueLimit()} which controls
-     * message back-pressure for server nodes. The default value for this parameter is {@link #DFLT_MSG_QUEUE_LIMIT}.
+     * message back-pressure for server nodes. The default value for this parameter is {@code 0}
+     * which means {@code unlimited}.
      *
-     * @param slowClientQueueLimit Slow cilent queue limit.
+     * @param slowClientQueueLimit Slow client queue limit.
      */
     public void setSlowClientQueueLimit(int slowClientQueueLimit) {
         this.slowClientQueueLimit = slowClientQueueLimit;
@@ -1171,20 +1172,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * Sets the minimum number of messages for this SPI, that are buffered
      * prior to sending.
-     * <p>
-     * Defaults to either {@code 512} or {@link IgniteSystemProperties#IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT}
-     * system property (if specified).
      *
      * @param minBufferedMsgCnt Minimum buffered message count.
+     * @deprecated Not used any more.
      */
     @IgniteSpiConfiguration(optional = true)
+    @Deprecated
     public void setMinimumBufferedMessageCount(int minBufferedMsgCnt) {
-        this.minBufferedMsgCnt = minBufferedMsgCnt;
+        // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public int getMinimumBufferedMessageCount() {
-        return minBufferedMsgCnt;
+        return 0;
     }
 
     /** {@inheritDoc} */
@@ -1248,7 +1249,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
         assertParameter(reconCnt > 0, "reconnectCnt > 0");
         assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
-        assertParameter(minBufferedMsgCnt >= 0, "minBufferedMsgCnt >= 0");
         assertParameter(connTimeout >= 0, "connTimeout >= 0");
         assertParameter(maxConnTimeout >= connTimeout, "maxConnTimeout >= connTimeout");
         assertParameter(sockWriteTimeout >= 0, "sockWriteTimeout >= 0");
@@ -1326,7 +1326,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
             log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
-            log.debug(configInfo("minBufferedMsgCnt", minBufferedMsgCnt));
             log.debug(configInfo("connTimeout", connTimeout));
             log.debug(configInfo("maxConnTimeout", maxConnTimeout));
             log.debug(configInfo("reconCnt", reconCnt));
@@ -1339,11 +1338,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
 
-        if (slowClientQueueLimit > 0 && msgQueueLimit > 0) {
-            if (slowClientQueueLimit >= msgQueueLimit) {
-                U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit. " +
-                    "Slow client queue limit will have no effect.");
-            }
+        if (slowClientQueueLimit > 0 && msgQueueLimit > 0 && slowClientQueueLimit >= msgQueueLimit) {
+            U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit " +
+                "(slow client queue limit will have no effect) [msgQueueLimit=" + msgQueueLimit +
+                    ", slowClientQueueLimit=" + slowClientQueueLimit + ']');
         }
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
@@ -1916,13 +1914,16 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
-                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [" +
-                        "srvNode=" + getSpiContext().localNode().id() +
+                    String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
+                        "the client will be dropped " +
+                        "(consider changing 'slowClientQueueLimit' configuration property) " +
+                        "[srvNode=" + getSpiContext().localNode().id() +
                         ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
-                    LT.warn(log, null, msg);
+                    U.quietAndWarn(
+                        log,
+                        msg);
 
                     getSpiContext().failNode(id, msg);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 1971d99..067ca41 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -168,7 +168,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * connection buffer.
      *
      * @return Flush frequency.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Connection buffer flush frequency.")
     public long getConnectionBufferFlushFrequency();
 
@@ -183,7 +185,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      *
      * @param connBufFlushFreq Flush frequency.
      * @see #getConnectionBufferSize()
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Sets connection buffer flush frequency.")
     public void setConnectionBufferFlushFrequency(long connBufFlushFreq);
 
@@ -193,7 +197,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * If set to {@code 0} connection buffer is disabled.
      *
      * @return Connection buffer size.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Connection buffer size.")
     public int getConnectionBufferSize();
 
@@ -238,7 +244,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * prior to sending.
      *
      * @return Minimum buffered message count.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Minimum buffered message count.")
     public int getMinimumBufferedMessageCount();
 


[30/33] incubator-ignite git commit: ignite-950: fixed memory leak

Posted by sb...@apache.org.
ignite-950: fixed memory leak


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

Branch: refs/heads/ignite-950
Commit: a088170ba796f4e8a93ad65c19f5f81d51633ca7
Parents: 42ec7d0
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 10:16:07 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 10:16:07 2015 +0300

----------------------------------------------------------------------
 .../optimized/ext/OptimizedMarshallerExt.java         |  5 +++++
 .../optimized/ext/OptimizedObjectInputStreamExt.java  |  7 +++++++
 .../optimized/ext/OptimizedObjectOutputStreamExt.java | 14 +++++++-------
 3 files changed, 19 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a088170b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
index da803d1..6bfe9c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExt.java
@@ -156,6 +156,11 @@ public class OptimizedMarshallerExt extends OptimizedMarshaller {
     }
 
     /** {@inheritDoc} */
+    @Override public void setPoolSize(int poolSize) {
+        OptimizedObjectStreamExtRegistry.poolSize(poolSize);
+    }
+
+    /** {@inheritDoc} */
     @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
         assert out != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a088170b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
index 7eb1fe6..1b75f57 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
@@ -69,6 +69,13 @@ public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
         return in.readByte();
     }
 
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        super.close();
+
+        metaHandler = null;
+    }
+
     /**
      * Checks whether the object has a field with name {@code fieldName}.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a088170b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
index 7ccb4b0..8ae8f9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectOutputStreamExt.java
@@ -74,6 +74,13 @@ public class OptimizedObjectOutputStreamExt extends OptimizedObjectOutputStream
             return null;
     }
 
+    /** {@inheritDoc} */
+    @Override public void close() throws IOException {
+        super.close();
+
+        metaHandler = null;
+    }
+
     /**
      *
      */
@@ -168,12 +175,5 @@ public class OptimizedObjectOutputStreamExt extends OptimizedObjectOutputStream
                 writeShort(footerLen);
             }
         }
-
-        /**
-         * Disable footer and indexing for the given Object.
-         */
-        private void disable() {
-            data = null;
-        }
     }
 }


[08/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973


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

Branch: refs/heads/ignite-950
Commit: bcfbb9236e860115dfc70da9f717aaba7cb9544c
Parents: 23512df d699faa
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 19 13:31:39 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 19 13:31:39 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |  2 +-
 modules/aop/pom.xml                             |  2 +-
 modules/aws/pom.xml                             |  2 +-
 modules/clients/pom.xml                         |  2 +-
 modules/cloud/pom.xml                           |  2 +-
 modules/codegen/pom.xml                         |  2 +-
 modules/core/pom.xml                            |  2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 42 +++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 35 ++++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 42 +++++++++++++------
 .../tcp/internal/TcpDiscoveryNode.java          | 18 ++++++++
 .../core/src/main/resources/ignite.properties   |  2 +-
 .../apache/ignite/internal/GridSelfTest.java    | 20 +++++----
 .../DataStreamerMultiThreadedSelfTest.java      |  3 ++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 ++++++++++++++++++--
 modules/extdata/p2p/pom.xml                     |  2 +-
 modules/extdata/uri/pom.xml                     |  2 +-
 modules/gce/pom.xml                             |  2 +-
 modules/geospatial/pom.xml                      |  2 +-
 modules/hadoop/pom.xml                          |  2 +-
 modules/hibernate/pom.xml                       |  2 +-
 modules/indexing/pom.xml                        |  2 +-
 modules/jcl/pom.xml                             |  2 +-
 modules/jta/pom.xml                             |  2 +-
 modules/log4j/pom.xml                           |  2 +-
 modules/mesos/pom.xml                           |  2 +-
 modules/rest-http/pom.xml                       |  2 +-
 modules/scalar-2.10/pom.xml                     |  2 +-
 modules/scalar/pom.xml                          |  2 +-
 modules/schedule/pom.xml                        |  2 +-
 modules/schema-import/pom.xml                   |  2 +-
 modules/slf4j/pom.xml                           |  2 +-
 modules/spark-2.10/pom.xml                      |  2 +-
 modules/spark/pom.xml                           |  2 +-
 modules/spring/pom.xml                          |  2 +-
 modules/ssh/pom.xml                             |  2 +-
 modules/tools/pom.xml                           |  2 +-
 modules/urideploy/pom.xml                       |  2 +-
 modules/visor-console-2.10/pom.xml              |  2 +-
 modules/visor-console/pom.xml                   |  2 +-
 modules/visor-plugins/pom.xml                   |  2 +-
 modules/web/pom.xml                             |  2 +-
 modules/yardstick/pom.xml                       |  2 +-
 pom.xml                                         |  2 +-
 44 files changed, 189 insertions(+), 89 deletions(-)
----------------------------------------------------------------------



[11/33] incubator-ignite git commit: #IGNITE-1034 - Corrected messages after code review.

Posted by sb...@apache.org.
#IGNITE-1034 - Corrected messages after code review.


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

Branch: refs/heads/ignite-950
Commit: e1b880030aae7ef9cfa6a812d17a97cf8055dfd5
Parents: 7a8b572
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 10:57:04 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 10:57:04 2015 -0700

----------------------------------------------------------------------
 .../ignite/spi/communication/tcp/TcpCommunicationSpi.java     | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1b88003/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index a540b7c..3997768 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1158,6 +1158,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * <p/>
      * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
      * those clients whose queue exceeded this limit.
+     * <p/>
+     * Usually this value should be set to the same value as {@link #getMessageQueueLimit()} which controls
+     * message back-pressure for server nodes. The default value for this parameter is {@link #DFLT_MSG_QUEUE_LIMIT}.
      *
      * @param slowClientQueueLimit Slow cilent queue limit.
      */
@@ -1913,8 +1916,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                    String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
+                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
                     LT.warn(log, null, msg);


[26/33] incubator-ignite git commit: Merge branch 'ignite-gg-10449' into ignite-sprint-6

Posted by sb...@apache.org.
Merge branch 'ignite-gg-10449' into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: de0930d3b92141a360ea6a2cc346035158572dcf
Parents: a4ce455 7c1ecc3
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 14:05:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 14:05:28 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 23 ++++++++++++++++++++
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 17 +++++++++++++++
 2 files changed, 40 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/de0930d3/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------


[07/33] incubator-ignite git commit: ignite-973 - swap manager fix + offheap processor signatures fix

Posted by sb...@apache.org.
ignite-973 - swap manager fix + offheap processor signatures fix


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

Branch: refs/heads/ignite-950
Commit: 23512df815c196f9a2292ad948b130380f123770
Parents: b23ea74
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 19 13:31:27 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 19 13:31:27 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 179 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 5 files changed, 220 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index d0d9049..f709e03 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -977,15 +977,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try offheap.
         if (offheapEnabled) {
-            byte[] val = offheap.remove(spaceName, part, key.value(cctx.cacheObjectContext(), false),
-                key.valueBytes(cctx.cacheObjectContext()));
-
-            if(val != null && cctx.config().isStatisticsEnabled())
-                cctx.cache().metrics0().onOffHeapRemove();
+            // TODO Pass closure c to offheap.remove and apply it before the actual remove.
+            byte[] val = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
 
             if (val != null) {
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapRemove();
+
                 if (c != null)
-                    c.apply(val); // Probably we should read value and apply closure before removing...
+                    c.apply(val);
 
                 return;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
index a99c4c0..81bf9f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.offheap;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.*;
@@ -101,7 +102,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Key bytes
      * @throws IgniteCheckedException If failed.
      */
-    private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
+    private byte[] keyBytes(KeyCacheObject key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
         assert key != null;
 
         return keyBytes != null ? keyBytes : marsh.marshal(key);
@@ -130,7 +131,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If offheap space contains value for the given key.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean contains(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public boolean contains(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -147,7 +148,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] get(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    @Nullable public byte[] get(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -166,7 +167,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Tuple where first value is pointer and second is value size.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, Object key,
+    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, KeyCacheObject key,
         byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -182,7 +183,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param keyBytes Key bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public void enableEviction(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public void enableEviction(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -201,7 +202,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public <T> T getValue(@Nullable String spaceName, int part, Object key, byte[] keyBytes,
+    @Nullable public <T> T getValue(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes,
         @Nullable ClassLoader ldr) throws IgniteCheckedException {
         byte[] valBytes = get(spaceName, part, key, keyBytes);
 
@@ -221,7 +222,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] remove(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    @Nullable public byte[] remove(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, keyBytes));
@@ -237,7 +238,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param valBytes Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public void put(@Nullable String spaceName, int part, Object key, byte[] keyBytes, byte[] valBytes)
+    public void put(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes, byte[] valBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -258,7 +259,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
index aadec74..98c8664 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.util;
 
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -173,13 +174,6 @@ public class GridDebug {
     }
 
     /**
-     * Dump collected data to stdout.
-     */
-    public static void dump() {
-        dump(que.get());
-    }
-
-    /**
      * Dumps given number of last events.
      *
      * @param n Number of last elements to dump.
@@ -204,7 +198,7 @@ public class GridDebug {
      * @param que Queue.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public static void dump(ConcurrentLinkedQueue<Item> que) {
+    public static void dump(Collection<Item> que) {
         if (que == null)
             return;
 
@@ -226,7 +220,7 @@ public class GridDebug {
      */
     public static String dumpWithStop(Object... x) {
         debug(x);
-        return dumpWithReset(null);
+        return dumpWithReset(null, null);
     }
 
     /**
@@ -235,16 +229,20 @@ public class GridDebug {
      * @return Empty string (useful for assertions like {@code assert x == 0 : D.dumpWithReset();} ).
      */
     public static String dumpWithReset() {
-        return dumpWithReset(new ConcurrentLinkedQueue<Item>());
+        return dumpWithReset(new ConcurrentLinkedQueue<Item>(), null);
     }
 
     /**
      * Dump existing queue to stdout and atomically replace it with given.
      *
      * @param q2 Queue.
+     * @param filter Filter for logged debug items.
      * @return Empty string.
      */
-    private static String dumpWithReset(@Nullable ConcurrentLinkedQueue<Item> q2) {
+    public static String dumpWithReset(
+        @Nullable ConcurrentLinkedQueue<Item> q2,
+        @Nullable IgnitePredicate<Item> filter
+    ) {
         ConcurrentLinkedQueue<Item> q;
 
         do {
@@ -255,7 +253,20 @@ public class GridDebug {
         }
         while (!que.compareAndSet(q, q2));
 
-        dump(q);
+        Collection<Item> col = null;
+
+        if (filter == null)
+            col = q;
+        else if (q != null) {
+            col = new ArrayList<>();
+
+            for (Item item : q) {
+                if (filter.apply(item))
+                    col.add(item);
+            }
+        }
+
+        dump(col);
 
         return "";
     }
@@ -281,7 +292,7 @@ public class GridDebug {
      */
     private static String formatEntry(long ts, String threadName, long threadId, Object... data) {
         return "<" + DEBUG_DATE_FMT.format(new Date(ts)) + "><~DBG~><" + threadName + " id:" + threadId + "> " +
-            Arrays.toString(data);
+            Arrays.deepToString(data);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
new file mode 100644
index 0000000..fc6c125
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -0,0 +1,179 @@
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.swapspace.inmemory.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ */
+public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
+
+        CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setSwapEnabled(true);
+        cacheCfg.setBackups(0);
+        cacheCfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
+        cacheCfg.setEvictionPolicy(null);
+        cacheCfg.setNearConfiguration(null);
+
+        cacheCfg.setSqlOnheapRowCacheSize(128);
+
+        cacheCfg.setIndexedTypes(
+            Integer.class, Integer.class
+        );
+
+        cacheCfg.setOffHeapMaxMemory(2000); // Small offheap for evictions from offheap to swap.
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictAndRemove() throws Exception {
+        final int KEYS_CNT = 3000;
+        final int THREADS_CNT = 256;
+
+        final IgniteCache<Integer,Integer> c = startGrid().cache(null);
+
+        for (int i = 0; i < KEYS_CNT; i++) {
+            c.put(i, i);
+
+            if ((i & 1) == 0)
+                c.localEvict(F.asList(i));
+        }
+
+        X.println("___ Cache loaded...");
+
+        final CyclicBarrier b = new CyclicBarrier(THREADS_CNT, new Runnable() {
+            @Override public void run() {
+                X.println("___ go!");
+            }
+        });
+
+        final AtomicInteger keys = new AtomicInteger(KEYS_CNT);
+
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
+            @Override public void run() {
+                Random rnd = new GridRandom();
+
+                try {
+                    b.await();
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteInterruptedException(e);
+                }
+                catch (BrokenBarrierException e) {
+                    throw new IllegalStateException(e);
+                }
+
+                while (keys.get() > 0) {
+                    int k = rnd.nextInt(KEYS_CNT);
+
+                    try {
+                        switch (rnd.nextInt(4)) {
+                            case 0:
+                                c.localEvict(F.asList(k));
+
+                                break;
+
+                            case 1:
+                                c.get(k);
+
+                                break;
+
+                            case 2:
+                                if (c.remove(k))
+                                    keys.decrementAndGet();
+
+                                break;
+
+                            case 3:
+                                c.query(new SqlFieldsQuery("select _val from Integer where _key between ? and ?")
+                                    .setArgs(k, k + 20).setLocal(true)).getAll();
+
+                                break;
+                        }
+                    }
+                    catch (CacheException e) {
+                        String msgStart = "Failed to get value for key:";
+
+                        for (Throwable th = e; th != null; th = th.getCause()) {
+                            String msg = th.getMessage();
+
+                            if (msg != null && msg.startsWith(msgStart)) {
+                                int dot = msg.indexOf('.', msgStart.length());
+
+                                assertTrue(dot != -1);
+
+                                final Integer failedKey = Integer.parseInt(msg.substring(msgStart.length(), dot).trim());
+
+                                X.println("___ failed key: " + failedKey);
+
+                                break;
+                            }
+                        }
+
+                        LT.warn(log, null, e.getMessage());
+
+                        return;
+                    }
+                }
+            }
+        }, THREADS_CNT);
+
+        try {
+            fut.get(60_000);
+
+            if (c.size(CachePeekMode.ALL) != 0)
+                fail("Not all keys removed.");
+
+            X.println("___ all keys removed");
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            X.println("___ timeout");
+            X.println("___ keys: " + keys.get());
+
+            keys.set(0);
+
+            fut.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index dee3078..b9205a9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -63,6 +63,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
         // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+        suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);


[21/33] incubator-ignite git commit: # ignite-sprint-6 minor

Posted by sb...@apache.org.
# ignite-sprint-6 minor


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

Branch: refs/heads/ignite-950
Commit: 6f5e676a2599a4e5441dc51f67763bdd10b2db24
Parents: 1cc0000
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 22 11:06:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 11:06:38 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 45 ++++++++++----------
 1 file changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f5e676a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index e255e08..b524e13 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -36,6 +36,7 @@ import java.io.*;
 import java.net.*;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
 
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.events.EventType.*;
@@ -79,7 +80,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     private volatile long topVer;
 
     /** Join error. Contains error what occurs on join process. */
-    private IgniteSpiException joinErr;
+    private final AtomicReference<IgniteSpiException> joinErr = new AtomicReference<>();
 
     /** Joined latch. */
     private final CountDownLatch joinLatch = new CountDownLatch(1);
@@ -171,8 +172,10 @@ class ClientImpl extends TcpDiscoveryImpl {
         try {
             joinLatch.await();
 
-            if (joinErr != null)
-                throw joinErr;
+            IgniteSpiException err = joinErr.get();
+
+            if (err != null)
+                throw err;
         }
         catch (InterruptedException e) {
             throw new IgniteSpiException("Thread has been interrupted.", e);
@@ -645,7 +648,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     private void joinError(IgniteSpiException err) {
         assert err != null;
 
-        joinErr = err;
+        joinErr.compareAndSet(null, err);
 
         joinLatch.countDown();
     }
@@ -1000,8 +1003,16 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (log.isDebugEnabled())
                             log.error("Reconnect error [join=" + join + ", timeout=" + timeout + ']', e);
 
-                        if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
+                        if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) {
+                            String msg = join ? "Failed to connect to cluster (consider increasing 'joinTimeout' " +
+                                "configuration  property) [joinTimeout=" + spi.joinTimeout + ", err=" + e + ']' :
+                                "Failed to reconnect to cluster (consider increasing 'networkTimeout' " +
+                                    "configuration  property) [networkTimeout=" + spi.netTimeout + ", err=" + e + ']';
+
+                            U.warn(log, msg);
+
                             throw e;
+                        }
                         else
                             U.warn(log, "Failed to reconnect to cluster (will retry): " + e);
                     }
@@ -1062,9 +1073,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 final Socket sock = joinTopology(false, spi.joinTimeout);
 
                 if (sock == null) {
-                    joinErr = new IgniteSpiException("Join process timed out.");
-
-                    joinLatch.countDown();
+                    joinError(new IgniteSpiException("Join process timed out."));
 
                     return;
                 }
@@ -1089,11 +1098,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (msg == JOIN_TIMEOUT) {
                         if (joinLatch.getCount() > 0) {
-                            joinErr = new IgniteSpiException("Join process timed out, did not receive response for " +
+                            joinError(new IgniteSpiException("Join process timed out, did not receive response for " +
                                 "join request (consider increasing 'joinTimeout' configuration property) " +
-                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + sock +']');
-
-                            joinLatch.countDown();
+                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + sock +']'));
 
                             break;
                         }
@@ -1159,9 +1166,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 err = spi.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
 
                             if (err != null) {
-                                joinErr = err;
-
-                                joinLatch.countDown();
+                                joinError(err);
 
                                 break;
                             }
@@ -1174,12 +1179,8 @@ class ClientImpl extends TcpDiscoveryImpl {
             finally {
                 U.closeQuiet(currSock);
 
-                if (joinLatch.getCount() > 0) {
-                    // This should not occurs.
-                    joinErr = new IgniteSpiException("Some error in join process.");
-
-                    joinLatch.countDown();
-                }
+                if (joinLatch.getCount() > 0)
+                    joinError(new IgniteSpiException("Some error in join process.")); // This should not occur.
 
                 if (reconnector != null) {
                     reconnector.cancel();
@@ -1297,7 +1298,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer, msg));
 
-                    joinErr = null;
+                    joinErr.set(null);;
 
                     joinLatch.countDown();
 


[29/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-6' into ignite-950

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-6' into ignite-950


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

Branch: refs/heads/ignite-950
Commit: 42ec7d0efdb6f4549520dd4726d08b93784c91bd
Parents: 9fb9679 285d790
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 08:30:39 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 08:30:39 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../processors/cache/GridCacheIoManager.java    |  64 ++++--
 .../GridCachePartitionExchangeManager.java      |  71 ++++---
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../GridDhtPartitionsExchangeFuture.java        |  49 +++--
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../ignite/internal/util/nio/GridNioServer.java |  64 +++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 104 +++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |  19 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  80 +++++---
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 ++++++++++++++++++
 .../GridCacheAbstractFailoverSelfTest.java      |   6 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++++++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   1 -
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 ++-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 36 files changed, 1043 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/42ec7d0e/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------


[12/33] incubator-ignite git commit: # fix for jta issue

Posted by sb...@apache.org.
# fix for jta issue


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

Branch: refs/heads/ignite-950
Commit: 20ac4b80ce90422268e90e9ef3cd9a84cbcfac2d
Parents: 4d9868a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jun 19 21:27:35 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jun 19 21:27:35 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java  |  1 +
 .../processors/cache/jta/GridCacheXAResource.java | 18 +++++++++---------
 .../processors/cache/GridCacheJtaSelfTest.java    |  2 +-
 3 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/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 3df45cb..ad4cf50 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
@@ -771,6 +771,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         }
                     }
                 }
+
                 if (skipped > 10)
                     fut.cleanUp();
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
index 505f558..442cdde 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
@@ -209,7 +209,14 @@ public final class GridCacheXAResource implements XAResource {
 
     /** {@inheritDoc} */
     @Override public int getTransactionTimeout() {
-        return (int)cacheTx.timeout();
+        return (int)(cacheTx.timeout() / 1000);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setTransactionTimeout(int i) {
+        cacheTx.timeout(i * 1000);
+
+        return true;
     }
 
     /** {@inheritDoc} */
@@ -217,7 +224,7 @@ public final class GridCacheXAResource implements XAResource {
         if (xar == this)
             return true;
 
-        if  (!(xar instanceof GridCacheXAResource))
+        if (!(xar instanceof GridCacheXAResource))
             return false;
 
         GridCacheXAResource other = (GridCacheXAResource)xar;
@@ -225,13 +232,6 @@ public final class GridCacheXAResource implements XAResource {
         return cctx == other.cctx;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean setTransactionTimeout(int i) {
-        cacheTx.timeout(i);
-
-        return true;
-    }
-
     /**
      *
      * @return {@code true} if jta was already committed or rolled back.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index cb7b4d6..b299f8d 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -152,7 +152,7 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
     @SuppressWarnings("ConstantConditions")
     public void testJtaTwoCaches() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-591");
-        
+
         UserTransaction jtaTx = jotm.getUserTransaction();
 
         IgniteEx ignite = grid(0);


[05/33] incubator-ignite git commit: # ignite-sprint-6 fixed NPE in comm spi unit tests

Posted by sb...@apache.org.
# ignite-sprint-6 fixed NPE in comm spi unit tests


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

Branch: refs/heads/ignite-950
Commit: 895771a79d8acca3baf06c23bf3f82483ec62d6f
Parents: ef4abeb
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 13:06:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 13:06:05 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/895771a7/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 84c1a57..bbb9b1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1458,8 +1458,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
+                boolean clientMode = Boolean.TRUE.equals(ignite.configuration().isClientMode());
+
                 IgniteBiInClosure<GridNioSession, Integer> queueSizeMonitor =
-                    !ignite.configuration().isClientMode() && slowClientQueueLimit > 0 ?
+                    !clientMode && slowClientQueueLimit > 0 ?
                     new CI2<GridNioSession, Integer>() {
                         @Override public void apply(GridNioSession ses, Integer qSize) {
                             checkClientQueueSize(ses, qSize);


[28/33] incubator-ignite git commit: # ignite-sprint-6 race in tcp communication connection, retry full partition map send

Posted by sb...@apache.org.
# ignite-sprint-6 race in tcp communication connection, retry full partition map send


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

Branch: refs/heads/ignite-950
Commit: 285d790ff0ce597fde65a778c381e59a4f1c89aa
Parents: 1605996
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 22 16:40:08 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 16:58:01 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 49 +++++++++++++++-----
 .../communication/tcp/TcpCommunicationSpi.java  | 12 +++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |  1 -
 3 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/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 9f18c98..7c780b0 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
@@ -1078,18 +1078,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 log.debug("Received message for finished future (will reply only to sender) [msg=" + msg +
                     ", fut=" + this + ']');
 
-            try {
-                ClusterNode n = cctx.node(nodeId);
-
-                if (n != null)
-                    sendAllPartitions(F.asList(n), exchId);
-            }
-            catch (IgniteCheckedException e) {
-                scheduleRecheck();
-
-                U.error(log, "Failed to send full partition map to node (will retry after timeout) [node=" + nodeId +
-                    ", exchangeId=" + exchId + ']', e);
-            }
+            sendAllPartitions(nodeId, cctx.gridConfig().getNetworkSendRetryCount());
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
@@ -1146,6 +1135,42 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @param nodeId Node ID.
+     * @param retryCnt Number of retries.
+     */
+    private void sendAllPartitions(final UUID nodeId, final int retryCnt) {
+        ClusterNode n = cctx.node(nodeId);
+
+        try {
+            if (n != null)
+                sendAllPartitions(F.asList(n), exchId);
+        }
+        catch (IgniteCheckedException e) {
+            if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) {
+                log.debug("Failed to send full partition map to node, node left grid " +
+                    "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']');
+
+                return;
+            }
+
+            if (retryCnt > 0) {
+                long timeout = cctx.gridConfig().getNetworkSendRetryDelay();
+
+                LT.error(log, e, "Failed to send full partition map to node (will retry after timeout) " +
+                    "[node=" + nodeId + ", exchangeId=" + exchId + ", timeout=" + timeout + ']');
+
+                cctx.time().addTimeoutObject(new GridTimeoutObjectAdapter(timeout) {
+                    @Override public void onTimeout() {
+                        sendAllPartitions(nodeId, retryCnt - 1);
+                    }
+                });
+            }
+            else
+                U.error(log, "Failed to send full partition map [node=" + n + ", exchangeId=" + exchId + ']', e);
+        }
+    }
+
+    /**
      * @param nodeId Sender node ID.
      * @param msg Full partition info.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1035ee5..addf243d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1748,6 +1748,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                                 assert old == null : "Client already created " +
                                         "[node=" + node + ", client=" + client0 + ", oldClient=" + old + ']';
+
+                                if (client0 instanceof GridTcpNioCommunicationClient) {
+                                    GridTcpNioCommunicationClient tcpClient = ((GridTcpNioCommunicationClient)client0);
+
+                                    if (tcpClient.session().closeTime() > 0 && clients.remove(nodeId, client0)) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Session was closed after client creation, will retry " +
+                                                "[node=" + node + ", client=" + client0 + ']');
+
+                                        client0 = null;
+                                    }
+                                }
                             }
                             else
                                 U.sleep(200);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index c4a0916..4062931 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -37,7 +37,6 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "messageQueueLimit", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "reconnectCount", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "selectorsCount", 0);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "minimumBufferedMessageCount", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectTimeout", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "maxConnectTimeout", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketWriteTimeout", -1);


[15/33] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: 415264e3e59d98e711fe37d7e4c12ef2416d9e7d
Parents: 03dc03d e0e1994
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 11:32:38 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 11:32:38 2015 -0700

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java  |  1 +
 .../processors/cache/jta/GridCacheXAResource.java | 18 +++++++++---------
 .../processors/cache/GridCacheJtaSelfTest.java    |  2 +-
 3 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------



[31/33] incubator-ignite git commit: ignite-950: test fixes, refactoring

Posted by sb...@apache.org.
ignite-950: test fixes, refactoring


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

Branch: refs/heads/ignite-950
Commit: d9f85e69aee657b3fb976ccb757c714efe78c106
Parents: a088170
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 10:46:24 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 10:46:24 2015 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |   2 +-
 .../communication/GridIoMessageFactory.java     |   4 +-
 .../cache/CacheIndexedObjectImpl.java           | 364 +++++++++++++++++++
 .../processors/cache/CacheObjectContext.java    |  12 +-
 .../cache/CacheOptimizedObjectImpl.java         | 364 -------------------
 .../cache/KeyCacheIndexedObjectImpl.java        | 104 ++++++
 .../cache/KeyCacheOptimizedObjectImpl.java      | 104 ------
 .../IgniteCacheObjectProcessorImpl.java         |  42 +--
 .../processors/query/GridQueryProcessor.java    | 119 ++----
 .../optimized/OptimizedMarshallerUtils.java     |   2 +-
 .../ext/OptimizedObjectInputStreamExt.java      |   2 +-
 .../ext/OptimizedMarshallerExtSelfTest.java     |  56 ++-
 12 files changed, 586 insertions(+), 589 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 4780ee4..e71e628 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -143,7 +143,7 @@ public class MessageCodeGenerator {
 
         MessageCodeGenerator gen = new MessageCodeGenerator(srcDir);
 
-        gen.generateAndWrite(CacheOptimizedObjectImpl.class);
+        gen.generateAndWrite(CacheIndexedObjectImpl.class);
 
 //        gen.generateAndWrite(GridDistributedLockRequest.class);
 //        gen.generateAndWrite(GridDistributedLockResponse.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/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 4c8fec5..29e778d 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
@@ -601,12 +601,12 @@ public class GridIoMessageFactory implements MessageFactory {
                 break;
 
             case 113:
-                msg = new CacheOptimizedObjectImpl();
+                msg = new CacheIndexedObjectImpl();
 
                 break;
 
             case 114:
-                msg = new KeyCacheOptimizedObjectImpl();
+                msg = new KeyCacheIndexedObjectImpl();
 
             // [-3..114] - this
             // [120..123] - DR

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
new file mode 100644
index 0000000..58e9c97
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
@@ -0,0 +1,364 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.jetbrains.annotations.*;
+import sun.misc.*;
+
+import java.io.*;
+import java.nio.*;
+
+/**
+ * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
+ * search and extraction without necessity to fully deserialize an object.
+ */
+public class CacheIndexedObjectImpl extends CacheObjectAdapter {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
+
+    /** */
+    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
+
+    /** */
+    protected int start;
+
+    /** */
+    protected int len;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public CacheIndexedObjectImpl() {
+       // No-op
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object.
+     * @param val Object.
+     */
+    public CacheIndexedObjectImpl(Object val) {
+        this(val, null, 0, 0);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object's serialized form.
+     * @param valBytes Object serialized to byte array.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public CacheIndexedObjectImpl(byte[] valBytes, int start, int len) {
+        this(null, valBytes, start, len);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl} with object's serialized form and value.
+     * @param val Object.
+     * @param valBytes Object serialized to byte array.
+     */
+    public CacheIndexedObjectImpl(Object val, byte[] valBytes) {
+        this(val, valBytes, 0, valBytes != null ? valBytes.length : 0);
+    }
+
+    /**
+     * Instantiates {@code CacheIndexedObjectImpl}.
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public CacheIndexedObjectImpl(Object val, byte[] valBytes, int start, int len) {
+        assert val != null || (valBytes != null && start >= 0 && len > 0);
+
+        this.val = val;
+        this.valBytes = valBytes;
+        this.start = start;
+        this.len = len;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
+        toMarshaledFormIfNeeded(ctx);
+
+        if (detached())
+            return valBytes;
+
+        byte[] arr = new byte[len];
+
+        U.arrayCopy(valBytes, start, arr, 0, len);
+
+        return arr;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        if (detached())
+            return this;
+
+        return detach();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
+        assert val != null || valBytes != null;
+
+        if (val == null && ctx.storeValue())
+            val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
+        assert val != null || valBytes != null;
+
+        toMarshaledFormIfNeeded(ctx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        // refer to GridIoMessageFactory.
+        return 113;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte type() {
+        return TYPE_OPTIMIZED;
+    }
+
+    /**
+     * Returns object's type ID.
+     *
+     * @return Type ID.
+     */
+    public int typeId() {
+        assert valBytes != null;
+
+        int typeId = UNSAFE.getInt(valBytes, BYTE_ARR_OFF + start + 1);
+
+        if (typeId == 0)
+            throw new IgniteException("Object's type ID wasn't written to cache.");
+
+        return typeId;
+    }
+
+    /**
+     * Checks whether a wrapped object has field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @param marsh Marshaller.
+     * @return {@code true} if has.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public boolean hasField(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
+        assert valBytes != null;
+
+        return marsh.hasField(fieldName, valBytes, start, len);
+    }
+
+    /**
+     * Searches and returns field if it exists.
+     *
+     * @param fieldName Field name.
+     * @param marsh Marshaller.
+     * @return Field.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public Object field(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
+        assert valBytes != null;
+
+        return marsh.readField(fieldName, valBytes, start, len, val != null ? val.getClass().getClassLoader() : null);
+    }
+
+    /**
+     * Deserializes wrapped object.
+     *
+     * @param ctx Cache context.
+     * @return Deserialized object.
+     */
+    public Object deserialize(CacheObjectContext ctx) {
+        if (val != null)
+            return val;
+
+        try {
+            assert valBytes != null;
+
+            Object val = ctx.processor().unmarshal(ctx, valBytes, start, len,
+                ctx.kernalContext().config().getClassLoader());
+
+            if (ctx.storeValue())
+                this.val = val;
+
+            return val;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to unmarshall object.", e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        super.writeExternal(out);
+
+        out.writeInt(start);
+        out.writeInt(len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        super.readExternal(in);
+
+        start = in.readInt();
+        len = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        if (!super.readFrom(buf, reader))
+            return false;
+
+        switch (reader.state()) {
+            case 1:
+                len = reader.readInt("len");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                start = reader.readInt("start");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!super.writeTo(buf, writer))
+            return false;
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 1:
+                if (!writer.writeInt("len", len))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeInt("start", start))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert false;
+
+        return super.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        assert false;
+
+        return super.equals(obj);
+    }
+
+    /**
+     * Detaches object.
+     *
+     * @return Detached object wrapped by {@code CacheIndexedObjectImpl}.
+     */
+    protected CacheIndexedObjectImpl detach() {
+        if (detached())
+            return this;
+
+        byte[] arr = new byte[len];
+
+        U.arrayCopy(valBytes, start, arr, 0, len);
+
+        return new CacheIndexedObjectImpl(arr, 0, len);
+    }
+
+    /**
+     * Checks whether the object is already detached or not.
+     *
+     * @return {@code true} if detached.
+     */
+    protected boolean detached() {
+        return start == 0 && len == valBytes.length;
+    }
+
+    /**
+     * Marshals {@link #val} to {@link #valBytes} if needed.
+     *
+     * @param ctx Cache object context.
+     * @throws IgniteCheckedException In case of error.
+     */
+    protected void toMarshaledFormIfNeeded(CacheObjectContext ctx) throws IgniteCheckedException {
+        if (valBytes == null) {
+            assert val != null;
+
+            valBytes = ctx.processor().marshal(ctx, val);
+
+            start = 0;
+            len = valBytes.length;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
index bf2cc07..4e5ca5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectContext.java
@@ -157,8 +157,8 @@ public class CacheObjectContext {
      * @return Unwrapped object.
      */
     private Object unwrapObject(Object obj) {
-        if (obj instanceof CacheOptimizedObjectImpl)
-            return ((CacheOptimizedObjectImpl)obj).deserialize(this);
+        if (obj instanceof CacheIndexedObjectImpl)
+            return ((CacheIndexedObjectImpl)obj).deserialize(this);
         else if (obj instanceof Map.Entry) {
             Map.Entry<Object, Object> entry = (Map.Entry<Object, Object>)obj;
 
@@ -166,16 +166,16 @@ public class CacheObjectContext {
 
             boolean unwrapped = false;
 
-            if (key instanceof CacheOptimizedObjectImpl) {
-                key = ((CacheOptimizedObjectImpl)key).deserialize(this);
+            if (key instanceof CacheIndexedObjectImpl) {
+                key = ((CacheIndexedObjectImpl)key).deserialize(this);
 
                 unwrapped = true;
             }
 
             Object val = entry.getValue();
 
-            if (val instanceof CacheOptimizedObjectImpl) {
-                val = ((CacheOptimizedObjectImpl)val).deserialize(this);
+            if (val instanceof CacheIndexedObjectImpl) {
+                val = ((CacheIndexedObjectImpl)val).deserialize(this);
 
                 unwrapped = true;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
deleted file mode 100644
index f07344c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOptimizedObjectImpl.java
+++ /dev/null
@@ -1,364 +0,0 @@
-/*
- * 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;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
-import org.apache.ignite.plugin.extensions.communication.*;
-import org.jetbrains.annotations.*;
-import sun.misc.*;
-
-import java.io.*;
-import java.nio.*;
-
-/**
- * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
- * search and extraction without necessity to fully deserialize an object.
- */
-public class CacheOptimizedObjectImpl extends CacheObjectAdapter {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** */
-    private static final Unsafe UNSAFE = GridUnsafe.unsafe();
-
-    /** */
-    private static final long BYTE_ARR_OFF = UNSAFE.arrayBaseOffset(byte[].class);
-
-    /** */
-    protected int start;
-
-    /** */
-    protected int len;
-
-    /**
-     * For {@link Externalizable}.
-     */
-    public CacheOptimizedObjectImpl() {
-       // No-op
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object.
-     * @param val Object.
-     */
-    public CacheOptimizedObjectImpl(Object val) {
-        this(val, null, 0, 0);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object's serialized form.
-     * @param valBytes Object serialized to byte array.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public CacheOptimizedObjectImpl(byte[] valBytes, int start, int len) {
-        this(null, valBytes, start, len);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl} with object's serialized form and value.
-     * @param val Object.
-     * @param valBytes Object serialized to byte array.
-     */
-    public CacheOptimizedObjectImpl(Object val, byte[] valBytes) {
-        this(val, valBytes, 0, valBytes != null ? valBytes.length : 0);
-    }
-
-    /**
-     * Instantiates {@code CacheOptimizedObjectImpl}.
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public CacheOptimizedObjectImpl(Object val, byte[] valBytes, int start, int len) {
-        assert val != null || (valBytes != null && start >= 0 && len > 0);
-
-        this.val = val;
-        this.valBytes = valBytes;
-        this.start = start;
-        this.len = len;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte[] valueBytes(CacheObjectContext ctx) throws IgniteCheckedException {
-        toMarshaledFormIfNeeded(ctx);
-
-        if (detached())
-            return valBytes;
-
-        byte[] arr = new byte[len];
-
-        U.arrayCopy(valBytes, start, arr, 0, len);
-
-        return arr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        if (detached())
-            return this;
-
-        return detach();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void finishUnmarshal(CacheObjectContext ctx, ClassLoader ldr) throws IgniteCheckedException {
-        assert val != null || valBytes != null;
-
-        if (val == null && ctx.storeValue())
-            val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void prepareMarshal(CacheObjectContext ctx) throws IgniteCheckedException {
-        assert val != null || valBytes != null;
-
-        toMarshaledFormIfNeeded(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        // refer to GridIoMessageFactory.
-        return 113;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte type() {
-        return TYPE_OPTIMIZED;
-    }
-
-    /**
-     * Returns object's type ID.
-     *
-     * @return Type ID.
-     */
-    public int typeId() {
-        assert valBytes != null;
-
-        int typeId = UNSAFE.getInt(valBytes, BYTE_ARR_OFF + start + 1);
-
-        if (typeId == 0)
-            throw new IgniteException("Object's type ID wasn't written to cache.");
-
-        return typeId;
-    }
-
-    /**
-     * Checks whether a wrapped object has field with name {@code fieldName}.
-     *
-     * @param fieldName Field name.
-     * @param marsh Marshaller.
-     * @return {@code true} if has.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public boolean hasField(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
-        assert valBytes != null;
-
-        return marsh.hasField(fieldName, valBytes, start, len);
-    }
-
-    /**
-     * Searches and returns field if it exists.
-     *
-     * @param fieldName Field name.
-     * @param marsh Marshaller.
-     * @return Field.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public Object field(String fieldName, OptimizedMarshallerExt marsh) throws IgniteCheckedException {
-        assert valBytes != null;
-
-        return marsh.readField(fieldName, valBytes, start, len, val != null ? val.getClass().getClassLoader() : null);
-    }
-
-    /**
-     * Deserializes wrapped object.
-     *
-     * @param ctx Cache context.
-     * @return Deserialized object.
-     */
-    public Object deserialize(CacheObjectContext ctx) {
-        if (val != null)
-            return val;
-
-        try {
-            assert valBytes != null;
-
-            Object val = ctx.processor().unmarshal(ctx, valBytes, start, len,
-                ctx.kernalContext().config().getClassLoader());
-
-            if (ctx.storeValue())
-                this.val = val;
-
-            return val;
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteException("Failed to unmarshall object.", e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void writeExternal(ObjectOutput out) throws IOException {
-        super.writeExternal(out);
-
-        out.writeInt(start);
-        out.writeInt(len);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-        super.readExternal(in);
-
-        start = in.readInt();
-        len = in.readInt();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
-        reader.setBuffer(buf);
-
-        if (!reader.beforeMessageRead())
-            return false;
-
-        if (!super.readFrom(buf, reader))
-            return false;
-
-        switch (reader.state()) {
-            case 1:
-                len = reader.readInt("len");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-            case 2:
-                start = reader.readInt("start");
-
-                if (!reader.isLastRead())
-                    return false;
-
-                reader.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
-        writer.setBuffer(buf);
-
-        if (!super.writeTo(buf, writer))
-            return false;
-
-        if (!writer.isHeaderWritten()) {
-            if (!writer.writeHeader(directType(), fieldsCount()))
-                return false;
-
-            writer.onHeaderWritten();
-        }
-
-        switch (writer.state()) {
-            case 1:
-                if (!writer.writeInt("len", len))
-                    return false;
-
-                writer.incrementState();
-
-            case 2:
-                if (!writer.writeInt("start", start))
-                    return false;
-
-                writer.incrementState();
-
-        }
-
-        return true;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte fieldsCount() {
-        return 3;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        assert false;
-
-        return super.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        assert false;
-
-        return super.equals(obj);
-    }
-
-    /**
-     * Detaches object.
-     *
-     * @return Detached object wrapped by {@code CacheOptimizedObjectImpl}.
-     */
-    protected CacheOptimizedObjectImpl detach() {
-        if (detached())
-            return this;
-
-        byte[] arr = new byte[len];
-
-        U.arrayCopy(valBytes, start, arr, 0, len);
-
-        return new CacheOptimizedObjectImpl(arr, 0, len);
-    }
-
-    /**
-     * Checks whether the object is already detached or not.
-     *
-     * @return {@code true} if detached.
-     */
-    protected boolean detached() {
-        return start == 0 && len == valBytes.length;
-    }
-
-    /**
-     * Marshals {@link #val} to {@link #valBytes} if needed.
-     *
-     * @param ctx Cache object context.
-     * @throws IgniteCheckedException In case of error.
-     */
-    protected void toMarshaledFormIfNeeded(CacheObjectContext ctx) throws IgniteCheckedException {
-        if (valBytes == null) {
-            assert val != null;
-
-            valBytes = ctx.processor().marshal(ctx, val);
-
-            start = 0;
-            len = valBytes.length;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
new file mode 100644
index 0000000..b63bca6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheIndexedObjectImpl.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
+ * search and extraction without necessity to fully deserialize an object.
+ */
+public class KeyCacheIndexedObjectImpl extends CacheIndexedObjectImpl implements KeyCacheObject {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     *
+     */
+    public KeyCacheIndexedObjectImpl() {
+        // No-op
+    }
+
+    /**
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     */
+    public KeyCacheIndexedObjectImpl(Object val, byte[] valBytes) {
+        super(val, valBytes);
+
+        assert val != null;
+    }
+
+    /**
+     * @param val Object.
+     * @param valBytes Object in a serialized form.
+     * @param start Object's start in the array.
+     * @param len Object's len in the array.
+     */
+    public KeyCacheIndexedObjectImpl(Object val, byte[] valBytes, int start, int len) {
+        super(val, valBytes, start, len);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        // refer to GridIoMessageFactory.
+        return 113;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean internal() {
+        assert val != null;
+
+        return val instanceof GridCacheInternal;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object deserialize(CacheObjectContext ctx) {
+        assert val != null;
+
+        return val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert val != null;
+
+        return val.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        if (!(obj instanceof KeyCacheIndexedObjectImpl))
+            return false;
+
+        KeyCacheIndexedObjectImpl other = (KeyCacheIndexedObjectImpl)obj;
+
+        return val.equals(other.val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
deleted file mode 100644
index 56e6fe2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/KeyCacheOptimizedObjectImpl.java
+++ /dev/null
@@ -1,104 +0,0 @@
-/*
- * 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;
-
-import org.jetbrains.annotations.*;
-
-/**
- * Cache object implementation for classes that support footer injection is their serialized form thus enabling fields
- * search and extraction without necessity to fully deserialize an object.
- */
-public class KeyCacheOptimizedObjectImpl extends CacheOptimizedObjectImpl implements KeyCacheObject {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /**
-     *
-     */
-    public KeyCacheOptimizedObjectImpl() {
-        // No-op
-    }
-
-    /**
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     */
-    public KeyCacheOptimizedObjectImpl(Object val, byte[] valBytes) {
-        super(val, valBytes);
-
-        assert val != null;
-    }
-
-    /**
-     * @param val Object.
-     * @param valBytes Object in a serialized form.
-     * @param start Object's start in the array.
-     * @param len Object's len in the array.
-     */
-    public KeyCacheOptimizedObjectImpl(Object val, byte[] valBytes, int start, int len) {
-        super(val, valBytes, start, len);
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Nullable @Override public <T> T value(CacheObjectContext ctx, boolean cpy) {
-        return (T)this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public CacheObject prepareForCache(CacheObjectContext ctx) {
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public byte directType() {
-        // refer to GridIoMessageFactory.
-        return 113;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean internal() {
-        assert val != null;
-
-        return val instanceof GridCacheInternal;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object deserialize(CacheObjectContext ctx) {
-        assert val != null;
-
-        return val;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        assert val != null;
-
-        return val.hashCode();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object obj) {
-        if (!(obj instanceof KeyCacheOptimizedObjectImpl))
-            return false;
-
-        KeyCacheOptimizedObjectImpl other = (KeyCacheOptimizedObjectImpl)obj;
-
-        return val.equals(other.val);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 748dd42..6ae958a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -196,10 +196,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
     protected KeyCacheObject toCacheKeyObject0(Object obj, boolean userObj) {
         if (!userObj)
-            return isFieldsIndexingEnabled(obj.getClass()) ? new KeyCacheOptimizedObjectImpl(obj, null) :
+            return isFieldsIndexingEnabled(obj.getClass()) ? new KeyCacheIndexedObjectImpl(obj, null) :
                 new KeyCacheObjectImpl(obj, null);
 
-        return isFieldsIndexingEnabled(obj.getClass()) ? new UserKeyCacheOptimizedObjectImpl(obj) :
+        return isFieldsIndexingEnabled(obj.getClass()) ? new UserKeyCacheIndexedObjectImpl(obj) :
             new UserKeyCacheObjectImpl(obj);
     }
 
@@ -239,7 +239,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                 return new CacheObjectImpl(null, bytes);
 
             case CacheObject.TYPE_OPTIMIZED:
-                return new CacheOptimizedObjectImpl(bytes, 0, bytes.length);
+                return new CacheIndexedObjectImpl(bytes, 0, bytes.length);
         }
 
         throw new IllegalArgumentException("Invalid object type: " + type);
@@ -271,10 +271,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         }
 
         if (!userObj)
-            return isFieldsIndexingEnabled(obj.getClass()) ? new CacheOptimizedObjectImpl(obj) :
+            return isFieldsIndexingEnabled(obj.getClass()) ? new CacheIndexedObjectImpl(obj) :
                 new CacheObjectImpl(obj, null);
 
-        return isFieldsIndexingEnabled(obj.getClass()) ? new UserCacheOptimizedObjectImpl(obj, null) :
+        return isFieldsIndexingEnabled(obj.getClass()) ? new UserCacheIndexedObjectImpl(obj, null) :
             new UserCacheObjectImpl(obj, null);
     }
 
@@ -313,8 +313,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public int typeId(Object obj) {
-        if (obj instanceof CacheOptimizedObjectImpl)
-            return ((CacheOptimizedObjectImpl)obj).typeId();
+        if (obj instanceof CacheIndexedObjectImpl)
+            return ((CacheIndexedObjectImpl)obj).typeId();
 
         return 0;
     }
@@ -336,11 +336,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public Object field(Object obj, String fieldName) {
-        if (obj instanceof CacheOptimizedObjectImpl) {
+        if (obj instanceof CacheIndexedObjectImpl) {
             assert optMarshExt != null;
 
             try {
-                return ((CacheOptimizedObjectImpl)obj).field(fieldName, optMarshExt);
+                return ((CacheIndexedObjectImpl)obj).field(fieldName, optMarshExt);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -352,11 +352,11 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
     /** {@inheritDoc} */
     @Override public boolean hasField(Object obj, String fieldName) {
-        if (obj instanceof CacheOptimizedObjectImpl) {
+        if (obj instanceof CacheIndexedObjectImpl) {
             assert optMarshExt != null;
 
             try {
-                return ((CacheOptimizedObjectImpl)obj).hasField(fieldName, optMarshExt);
+                return ((CacheIndexedObjectImpl)obj).hasField(fieldName, optMarshExt);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException(e);
@@ -480,14 +480,14 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
      * Wraps value provided by user, must be serialized before stored in cache.
      * Used by classes that support fields indexing. Refer to {@link #isFieldsIndexingEnabled(Class)}.
      */
-    private static class UserCacheOptimizedObjectImpl extends CacheOptimizedObjectImpl {
+    private static class UserCacheIndexedObjectImpl extends CacheIndexedObjectImpl {
         /** */
         private static final long serialVersionUID = 0L;
 
         /**
          *
          */
-        public UserCacheOptimizedObjectImpl() {
+        public UserCacheIndexedObjectImpl() {
             //No-op.
         }
 
@@ -495,7 +495,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
          * @param val Value.
          * @param valBytes Value bytes.
          */
-        public UserCacheOptimizedObjectImpl(Object val, byte[] valBytes) {
+        public UserCacheIndexedObjectImpl(Object val, byte[] valBytes) {
             super(val, valBytes);
         }
 
@@ -516,10 +516,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
                     Object val = this.val != null && ctx.processor().immutable(this.val) ? this.val :
                         ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
 
-                    return new CacheOptimizedObjectImpl(val, valBytes, start, len);
+                    return new CacheIndexedObjectImpl(val, valBytes, start, len);
                 }
 
-                return new CacheOptimizedObjectImpl(null, valBytes, start, len);
+                return new CacheIndexedObjectImpl(null, valBytes, start, len);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to marshal object: " + val, e);
@@ -531,21 +531,21 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
      * Wraps key provided by user, must be serialized before stored in cache.
      * Used by classes that support fields indexing. Refer to {@link #isFieldsIndexingEnabled(Class)}.
      */
-    private static class UserKeyCacheOptimizedObjectImpl extends KeyCacheOptimizedObjectImpl {
+    private static class UserKeyCacheIndexedObjectImpl extends KeyCacheIndexedObjectImpl {
         /** */
         private static final long serialVersionUID = 0L;
 
         /**
          *
          */
-        public UserKeyCacheOptimizedObjectImpl() {
+        public UserKeyCacheIndexedObjectImpl() {
             //No-op.
         }
 
         /**
          * @param key Key.
          */
-        UserKeyCacheOptimizedObjectImpl(Object key) {
+        UserKeyCacheIndexedObjectImpl(Object key) {
             super(key, null);
         }
 
@@ -560,10 +560,10 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
                     Object val = ctx.processor().unmarshal(ctx, valBytes, start, len, ldr);
 
-                    return new KeyCacheOptimizedObjectImpl(val, valBytes, start, len);
+                    return new KeyCacheIndexedObjectImpl(val, valBytes, start, len);
                 }
 
-                return new KeyCacheOptimizedObjectImpl(val, valBytes, start, len);
+                return new KeyCacheIndexedObjectImpl(val, valBytes, start, len);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to marshal object: " + val, e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e5800de..f51a0bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -47,6 +47,7 @@ import java.util.concurrent.*;
 import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.IgniteComponentType.*;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.*;
+import static org.apache.ignite.internal.processors.query.GridQueryProcessor.PropertyType.*;
 
 /**
  * Indexing processor.
@@ -141,12 +142,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     TypeId typeId;
 
                     if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
-                        processPortableMeta(meta, desc);
+                        processCacheTypeMeta(meta, desc, PORTABLE_PROPERTY);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
                     }
                     else if (ctx.cacheObjects().enableFieldsIndexing(valCls)) {
-                        processIndexedFieldsMeta(meta, desc);
+                        processCacheTypeMeta(meta, desc, INDEXED_FIELDS_PROPERTY);
 
                         typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(valCls.getName()));
                     }
@@ -459,7 +460,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             TypeId id;
 
             boolean portableVal = ctx.cacheObjects().isPortableObject(val);
-            boolean indexedFieldsVal = val instanceof CacheOptimizedObjectImpl;
+            boolean indexedFieldsVal = val instanceof CacheIndexedObjectImpl;
 
             if (portableVal || indexedFieldsVal) {
                 int typeId = ctx.cacheObjects().typeId(val);
@@ -482,7 +483,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     "(multiple classes with same simple name are stored in the same cache) " +
                     "[expCls=" + desc.valueClass().getName() + ", actualCls=" + valCls.getName() + ']');
 
-            if (!(key instanceof CacheOptimizedObjectImpl) && !ctx.cacheObjects().isPortableObject(key)) {
+            if (!(key instanceof CacheIndexedObjectImpl) && !ctx.cacheObjects().isPortableObject(key)) {
                 Class<?> keyCls = key.value(coctx, false).getClass();
 
                 if (!desc.keyClass().isAssignableFrom(keyCls))
@@ -1234,91 +1235,21 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Processes declarative metadata for portable object.
+     * Processes declarative metadata.
      *
      * @param meta Declared metadata.
      * @param d Type descriptor.
+     * @param propertyType PropertyType.
      * @throws IgniteCheckedException If failed.
      */
-    private void processPortableMeta(CacheTypeMetadata meta, TypeDescriptor d)
+    private void processCacheTypeMeta(CacheTypeMetadata meta, TypeDescriptor d, PropertyType propertyType)
         throws IgniteCheckedException {
-        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            d.addProperty(prop, false);
-
-            String idxName = prop.name() + "_idx";
-
-            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
-
-            d.addFieldToIndex(idxName, prop.name(), 0, false);
-        }
-
-        for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            d.addProperty(prop, false);
-
-            String idxName = prop.name() + "_idx";
-
-            d.addIndex(idxName, idx.isGeometryClass(prop.type()) ? GEO_SPATIAL : SORTED);
-
-            d.addFieldToIndex(idxName, prop.name(), 0, true);
-        }
-
-        for (String txtIdx : meta.getTextFields()) {
-            PortableProperty prop = buildPortableProperty(txtIdx, String.class);
-
-            d.addProperty(prop, false);
-
-            d.addFieldToTextIndex(prop.name());
-        }
-
-        Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps = meta.getGroups();
-
-        if (grps != null) {
-            for (Map.Entry<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> entry : grps.entrySet()) {
-                String idxName = entry.getKey();
-
-                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> idxFields = entry.getValue();
-
-                int order = 0;
-
-                for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    PortableProperty prop = buildPortableProperty(idxField.getKey(), idxField.getValue().get1());
-
-                    d.addProperty(prop, false);
-
-                    Boolean descending = idxField.getValue().get2();
-
-                    d.addFieldToIndex(idxName, prop.name(), order, descending != null && descending);
-
-                    order++;
-                }
-            }
-        }
-
-        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            PortableProperty prop = buildPortableProperty(entry.getKey(), entry.getValue());
-
-            if (!d.props.containsKey(prop.name()))
-                d.addProperty(prop, false);
-        }
-    }
-
-    /**
-     * Processes declarative metadata for object that has fields information in its serialized form.
-     *
-     * @param meta Declared metadata.
-     * @param d Type descriptor.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void processIndexedFieldsMeta(CacheTypeMetadata meta, TypeDescriptor d)
-        throws IgniteCheckedException {
-        //TODO: IGNITE-950, refactor. The code is similar to portable properties ones.
+        assert propertyType != CLASS_PROPERTY;
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1330,7 +1261,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1342,7 +1275,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(txtIdx, String.class);
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(txtIdx, String.class) :
+                buildIndexedFieldsProperty(txtIdx, String.class);
 
             d.addProperty(prop, false);
 
@@ -1360,8 +1295,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 int order = 0;
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    IndexedFieldsProperty prop = buildIndexedFieldsProperty(idxField.getKey(),
-                        idxField.getValue().get1());
+                    Property prop = propertyType == PORTABLE_PROPERTY ?
+                        buildPortableProperty(idxField.getKey(), idxField.getValue().get1()) :
+                        buildIndexedFieldsProperty(idxField.getKey(), idxField.getValue().get1());
 
                     d.addProperty(prop, false);
 
@@ -1375,7 +1311,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            IndexedFieldsProperty prop = buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
+            Property prop = propertyType == PORTABLE_PROPERTY ?
+                buildPortableProperty(entry.getKey(), entry.getValue()) :
+                buildIndexedFieldsProperty(entry.getKey(), entry.getValue());
 
             if (!d.props.containsKey(prop.name()))
                 d.addProperty(prop, false);
@@ -2247,4 +2185,13 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             return S.toString(TypeName.class, this);
         }
     }
+
+    /**
+     *
+     */
+    enum PropertyType {
+        CLASS_PROPERTY,
+        PORTABLE_PROPERTY,
+        INDEXED_FIELDS_PROPERTY
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index 2f42e8d..7a7ee69 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -248,7 +248,7 @@ public class OptimizedMarshallerUtils {
         if (obj == null)
             return false;
 
-        if (obj instanceof CacheOptimizedObjectImpl ||
+        if (obj instanceof CacheIndexedObjectImpl ||
             obj instanceof Map.Entry ||
             obj instanceof Collection ||
             obj instanceof Map ||

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
index 1b75f57..66544a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/ext/OptimizedObjectInputStreamExt.java
@@ -125,7 +125,7 @@ public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
 
             if (in.readByte() == SERIALIZABLE && metaHandler.metadata(in.readInt()) != null)
                 //Do we need to make a copy of array?
-                field = (F)new CacheOptimizedObjectImpl(in.array(), range.start, range.len);
+                field = (F)new CacheIndexedObjectImpl(in.array(), range.start, range.len);
             else {
                 in.position(range.start);
                 field = (F)readObject();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d9f85e69/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
index 2526de7..c1f07ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/ext/OptimizedMarshallerExtSelfTest.java
@@ -22,24 +22,44 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import java.util.concurrent.*;
+
 /**
  * Optimized marshaller self test.
  */
 @GridCommonTest(group = "Marshaller")
 public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest {
+    /** */
+    private static ConcurrentHashMap<Integer, OptimizedObjectMetadata> META_BUF = new ConcurrentHashMap<>();
+
+    /** */
+    private static final OptimizedMarshallerExtMetaHandler META_HANDLER = new OptimizedMarshallerExtMetaHandler() {
+        @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
+            META_BUF.putIfAbsent(typeId, meta);
+        }
+
+        @Override public OptimizedObjectMetadata metadata(int typeId) {
+            return META_BUF.get(typeId);
+        }
+    };
+
     /** {@inheritDoc} */
     @Override protected Marshaller marshaller() {
-        return new OptimizedMarshallerExt(false);
+        return new InternalMarshaller(false);
     }
 
     /**
      * @throws Exception In case of error.
      */
     public void testHasField() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(TestObject.class));
 
+        assertEquals(1, META_BUF.size());
+
         TestObject testObj = new TestObject("World", 50);
 
         byte[] arr = marsh.marshal(testObj);
@@ -54,9 +74,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
      * @throws Exception In case of error.
      */
     public void testReadField() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(TestObject.class));
+        assertEquals(1, META_BUF.size());
 
         TestObject testObj = new TestObject("World", 50);
 
@@ -75,11 +98,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
 
         // Add metadata for the enclosed object.
         assertTrue(marsh.enableFieldsIndexing(TestObject2.class));
+        assertEquals(2, META_BUF.size());
 
         arr = marsh.marshal(testObj);
 
         // Must be returned in a wrapped form, since metadata was added enabling the footer.
-        CacheOptimizedObjectImpl cacheObject = marsh.readField("o2", arr, 0, arr.length, null);
+        CacheIndexedObjectImpl cacheObject = marsh.readField("o2", arr, 0, arr.length, null);
 
         arr = cacheObject.valueBytes(null);
 
@@ -92,9 +116,12 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
      * @throws Exception In case of error.
      */
     public void testHandles() throws Exception {
+        META_BUF.clear();
+
         OptimizedMarshallerExt marsh = (OptimizedMarshallerExt)OptimizedMarshallerExtSelfTest.marsh;
 
         assertTrue(marsh.enableFieldsIndexing(SelfLinkObject.class));
+        assertEquals(1, META_BUF.size());
 
         SelfLinkObject selfLinkObject = new SelfLinkObject();
         selfLinkObject.str1 = "Hello, world!";
@@ -107,7 +134,7 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
 
         assertEquals(selfLinkObject.str1, str2);
 
-        CacheOptimizedObjectImpl cacheObj = marsh.readField("link", arr, 0, arr.length, null);
+        CacheIndexedObjectImpl cacheObj = marsh.readField("link", arr, 0, arr.length, null);
 
         arr = cacheObj.valueBytes(null);
 
@@ -116,6 +143,29 @@ public class OptimizedMarshallerExtSelfTest extends OptimizedMarshallerSelfTest
         assertEquals(selfLinkObject, selfLinkObject2);
     }
 
+    private static class InternalMarshaller extends OptimizedMarshallerExt {
+        /**
+         * Constructor.
+         */
+        public InternalMarshaller() {
+        }
+
+        /**
+         * Constructor.
+         * @param requireSer Requires serialiazable.
+         */
+        public InternalMarshaller(boolean requireSer) {
+            super(requireSer);
+
+            super.setMetadataHandler(META_HANDLER);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void setMetadataHandler(OptimizedMarshallerExtMetaHandler metaHandler) {
+            // No-op
+        }
+    }
+
     /** */
     private static class TestObject2 {
         /** */


[25/33] incubator-ignite git commit: #IGNITE-GG-10449 Fixed license.

Posted by sb...@apache.org.
#IGNITE-GG-10449 Fixed license.


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

Branch: refs/heads/ignite-950
Commit: 7c1ecc3dd2e240e48958c8fb1fd812a6df7e92b5
Parents: a3d8e60
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 13:15:55 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 13:15:55 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapEvictQueryTest.java    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c1ecc3d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index 72bfcaa..45d744e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -1,3 +1,20 @@
+/*
+ * 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;
 
 import org.apache.ignite.*;


[18/33] incubator-ignite git commit: #ignite-973 - fix for IGNITE-1041

Posted by sb...@apache.org.
#ignite-973 - fix for IGNITE-1041


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

Branch: refs/heads/ignite-950
Commit: ced62b8708952abfabc4926cfbb28c424d1ce651
Parents: de27ff1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:35:43 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:35:43 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |  2 ++
 .../query/h2/twostep/GridMapQueryExecutor.java  | 23 ++++++++++----------
 2 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ced62b87/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c162ae6..06c0961 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -1354,6 +1354,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Schema rmv = schemas.remove(schema);
 
         if (rmv != null) {
+            mapQryExec.onCacheStop(ccfg.getName());
+
             try {
                 dropSchema(schema);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ced62b87/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index ba95d43..0f38353 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -124,18 +124,6 @@ public class GridMapQueryExecutor {
             }
         }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
 
-        // Drop group reservations for dead caches.
-        ctx.event().addLocalEventListener(new GridLocalEventListener() {
-            @Override public void onEvent(Event evt) {
-                String cacheName = ((CacheEvent)evt).cacheName();
-
-                for (T2<String,AffinityTopologyVersion> grpKey : reservations.keySet()) {
-                    if (F.eq(grpKey.get1(), cacheName))
-                        reservations.remove(grpKey);
-                }
-            }
-        }, EventType.EVT_CACHE_STOPPED);
-
         ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object msg) {
                 if (!busyLock.enterBusy())
@@ -599,6 +587,17 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * @param cacheName Cache name.
+     */
+    public void onCacheStop(String cacheName) {
+        // Drop group reservations.
+        for (T2<String,AffinityTopologyVersion> grpKey : reservations.keySet()) {
+            if (F.eq(grpKey.get1(), cacheName))
+                reservations.remove(grpKey);
+        }
+    }
+
+    /**
      *
      */
     private class QueryResults {


[13/33] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: e0e1994dd51d81b0d7d47000db1678f0268c0bfd
Parents: 20ac4b8 e1b8800
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jun 19 21:28:12 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jun 19 21:28:12 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  69 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 ++++-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 19 files changed, 495 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[03/33] incubator-ignite git commit: # ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test

Posted by sb...@apache.org.
# ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test


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

Branch: refs/heads/ignite-950
Commit: 462495f2977668ae9353adecef554c4f15dd70f3
Parents: 44bbece
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 10:12:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 11:47:14 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |  8 +--
 .../discovery/GridDiscoveryManager.java         | 19 ++++--
 .../continuous/CacheContinuousQueryHandler.java |  8 +++
 .../ignite/internal/util/nio/GridNioServer.java | 13 ++--
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  4 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  6 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 20 +++---
 .../tcp/TcpCommunicationSpiMBean.java           | 11 ++++
 .../ignite/spi/discovery/DiscoverySpi.java      |  3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 12 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 12 +++-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java | 18 +++++
 ...ridFailFastNodeFailureDetectionSelfTest.java | 17 ++++-
 .../IgniteSlowClientDetectionSelfTest.java      | 69 +++++++++++++++++++-
 .../testframework/GridSpiTestContext.java       |  4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 18 files changed, 189 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 885d52c..40a5ea5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -480,12 +480,12 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.io().messageFactory();
                     }
 
-                    @Override public boolean tryFailNode(UUID nodeId) {
-                        return ctx.discovery().tryFailNode(nodeId);
+                    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
+                        return ctx.discovery().tryFailNode(nodeId, warning);
                     }
 
-                    @Override public void failNode(UUID nodeId) {
-                        ctx.discovery().failNode(nodeId);
+                    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+                        ctx.discovery().failNode(nodeId, warning);
                     }
 
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 717cdf3..1e4b972 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -386,9 +386,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
                 else {
-                    minorTopVer = 0;
+                    if (type != EVT_NODE_SEGMENTED) {
+                        minorTopVer = 0;
 
-                    verChanged = true;
+                        verChanged = true;
+                    }
+                    else
+                        verChanged = false;
                 }
 
                 AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
@@ -1481,15 +1485,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning message to be shown on all nodes.
      * @return Whether node is failed.
      */
-    public boolean tryFailNode(UUID nodeId) {
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return false;
 
         try {
             if (!getSpi().pingNode(nodeId)) {
-                getSpi().failNode(nodeId);
+                getSpi().failNode(nodeId, warning);
 
                 return true;
             }
@@ -1503,13 +1508,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID to fail.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public void failNode(UUID nodeId) {
+    public void failNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return;
 
         try {
-            getSpi().failNode(nodeId);
+            getSpi().failNode(nodeId, warning);
         }
         finally {
             busyLock.leaveBusy();
@@ -1520,6 +1526,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.
+     * @param discoCache Discovery cache.
      * @return {@code True} if topology was updated.
      */
     private boolean updateTopologyVersionIfGreater(AffinityTopologyVersion updated, DiscoCache discoCache) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index ad78b92..ff2905f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.managers.deployment.*;
@@ -226,6 +227,13 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
                             ctx.continuous().addNotification(nodeId, routineId, evt.entry(), topic, sync, true);
                         }
+                        catch (ClusterTopologyCheckedException ex) {
+                            IgniteLogger log = ctx.log(getClass());
+
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send event notification to node, node left cluster " +
+                                    "[node=" + nodeId + ", err=" + ex + ']');
+                        }
                         catch (IgniteCheckedException ex) {
                             U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index b9d246a..24e1e08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -505,13 +505,18 @@ public class GridNioServer<T> {
     public GridNioFuture<GridNioSession> createSession(final SocketChannel ch,
         @Nullable Map<Integer, ?> meta) {
         try {
-            ch.configureBlocking(false);
+            if (!closed) {
+                ch.configureBlocking(false);
 
-            NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, false, meta);
+                NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, false, meta);
 
-            offerBalanced(req);
+                offerBalanced(req);
 
-            return req;
+                return req;
+            }
+            else
+                return new GridNioFinishedFuture<>(
+                    new IgniteCheckedException("Failed to create session, server is stopped."));
         }
         catch (IOException e) {
             return new GridNioFinishedFuture<>(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 18191a1..5e557bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -754,12 +754,12 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public boolean tryFailNode(UUID nodeId) {
+        @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
             return false;
         }
 
         /** {@inheritDoc} */
-        @Override public void failNode(UUID nodeId) {
+        @Override public void failNode(UUID nodeId, @Nullable String warning) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index a655a73..611702b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -307,14 +307,16 @@ public interface IgniteSpiContext {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      * @return If node was failed.
      */
-    public boolean tryFailNode(UUID nodeId);
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param c Timeout object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 538e9a8..84c1a57 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1153,15 +1153,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return msgQueueLimit;
     }
 
-    /**
-     * Gets slow client queue limit.
-     * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
-     * those clients whose queue exceeded this limit.
-     *
-     * @return Slow client queue limit.
-     */
-    public int getSlowClientQueueLimit() {
+    /** {@inheritDoc} */
+    @Override public int getSlowClientQueueLimit() {
         return slowClientQueueLimit;
     }
 
@@ -1923,10 +1916,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
+
+                    LT.warn(log, null, msg);
 
-                    getSpiContext().failNode(id);
+                    getSpiContext().failNode(id, msg);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index fe4f581..1971d99 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -268,4 +268,15 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Maximum number of unacknowledged messages.")
     public int getUnacknowledgedMessagesBufferSize();
+
+    /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    @MXBeanDescription("Slow client queue limit.")
+    public int getSlowClientQueueLimit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index b952087..11a18b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -151,8 +151,9 @@ public interface DiscoverySpi extends IgniteSpi {
      * Initiates failure of provided node.
      *
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * Whether or not discovery is started in client mode.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index fef6f4f..e255e08 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -335,13 +335,15 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = rmtNodes.get(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -1432,6 +1434,14 @@ class ClientImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
+                if (msg.warning() != null) {
+                    ClusterNode creatorNode = rmtNodes.get(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 8eb82ac..2458f85 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -584,13 +584,15 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = ring.node(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -3409,6 +3411,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         worker.interrupt();
                 }
 
+                if (msg.warning() != null && !msg.creatorNodeId().equals(getLocalNodeId())) {
+                    ClusterNode creatorNode = ring.node(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 94097c9..ace917f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -175,8 +175,9 @@ abstract class TcpDiscoveryImpl {
 
     /**
      * @param nodeId Node id.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public abstract void failNode(UUID nodeId);
+    public abstract void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param gridName Grid name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index c36ac76..1d1916a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -373,8 +373,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        impl.failNode(nodeId);
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+        impl.failNode(nodeId, warning);
     }
 
     /** {@inheritDoc} */
@@ -385,7 +385,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** {@inheritDoc} */
     @Override public boolean isClientMode() {
         if (impl == null)
-            throw new IllegalStateException("TcpDiscoverySpi has not started");
+            throw new IllegalStateException("TcpDiscoverySpi has not started.");
 
         return impl instanceof ClientImpl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
index 8cb8414..93ecdaa 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -37,6 +38,9 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     /** Internal order of the failed node. */
     private final long order;
 
+    /** */
+    private String warning;
+
     /**
      * Constructor.
      *
@@ -55,6 +59,20 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
+     * @param warning Warning message to be shown on all nodes.
+     */
+    public void warning(String warning) {
+        this.warning = warning;
+    }
+
+    /**
+     * @return Warning message to be shown on all nodes.
+     */
+    @Nullable public String warning() {
+        return warning;
+    }
+
+    /**
      * Gets ID of the failed node.
      *
      * @return ID of the failed node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 992d7bf..238115d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -50,7 +50,12 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(IP_FINDER);
-        disco.setHeartbeatFrequency(10000);
+        disco.setHeartbeatFrequency(10_000);
+
+        // Set parameters for fast ping failure.
+        disco.setSocketTimeout(100);
+        disco.setNetworkTimeout(100);
+        disco.setReconnectCount(2);
 
         cfg.setDiscoverySpi(disco);
 
@@ -66,8 +71,6 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testFailFast() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-933");
-
         startGridsMultiThreaded(5);
 
         final CountDownLatch failLatch = new CountDownLatch(4);
@@ -87,6 +90,8 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         Ignite ignite1 = ignite(0);
         Ignite ignite2 = ignite(1);
 
+        final CountDownLatch evtLatch = new CountDownLatch(1);
+
         ignite1.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
                 respond(rcvMsg);
@@ -95,12 +100,18 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         ignite2.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
+                evtLatch.countDown();
+
                 respond(rcvMsg);
             }
         });
 
         ignite1.message(ignite1.cluster().forRemotes()).send(null, "Message");
 
+        evtLatch.await(); // Wait when connection is established.
+
+        log.info("Fail node: " + ignite1.cluster().localNode());
+
         failNode(ignite1);
 
         assert failLatch.await(1000, MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 09b4215..27c2a61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -19,24 +19,37 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.event.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
 
 /**
  *
  */
 public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
-
+    /** */
     public static final String PARTITIONED = "partitioned";
 
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @return Node count.
      */
@@ -48,6 +61,8 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
         if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
             cfg.setClientMode(true);
 
@@ -66,7 +81,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(nodeCount());
+        startGrids(nodeCount());
     }
 
     /** {@inheritDoc} */
@@ -82,6 +97,45 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     public void testSlowClient() throws Exception {
         final IgniteEx slowClient = grid(nodeCount() - 1);
 
+        final ClusterNode slowClientNode = slowClient.localNode();
+
+        final CountDownLatch evtSegmentedLatch = new CountDownLatch(1);
+
+        slowClient.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_SEGMENTED);
+
+                DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+                assertEquals(slowClientNode, evt0.eventNode());
+                assertEquals(5L, evt0.topologyVersion());
+
+                evtSegmentedLatch.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        final CountDownLatch evtFailedLatch = new CountDownLatch(nodeCount() - 1);
+
+        for (int i = 0; i < nodeCount() - 1; i++) {
+            grid(i).events().localListen(new IgnitePredicate<Event>() {
+                @Override public boolean apply(Event evt) {
+                    assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_FAILED);
+
+                    DiscoveryEvent evt0 = (DiscoveryEvent) evt;
+
+                    assertEquals(slowClientNode, evt0.eventNode());
+                    assertEquals(6L, evt0.topologyVersion());
+                    assertEquals(4, evt0.topologyNodes().size());
+
+                    evtFailedLatch.countDown();
+
+                    return false;
+                }
+            }, EventType.EVT_NODE_FAILED);
+        }
+
         assertTrue(slowClient.cluster().localNode().isClient());
 
         IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
@@ -109,14 +163,23 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < 100; i++)
             cache0.put(0, new byte[10 * 1024]);
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
             }
         }, getTestTimeout());
+
+        assertTrue(wait);
+
+        assertTrue("Failed to wait for client failed event", evtFailedLatch.await(5000, MILLISECONDS));
+        assertTrue("Failed to wait for client segmented event", evtSegmentedLatch.await(5000, MILLISECONDS));
     }
 
+    /**
+     *
+     */
     private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        /** {@inheritDoc} */
         @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
             System.out.println(">>>> Received update: " + iterable);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index c20ff2e..08268af 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -497,12 +497,12 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tryFailNode(UUID nodeId) {
+    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index b4977ce..2d14728 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -91,6 +91,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridMessageListenSelfTest.class);
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
+        suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
 
         return suite;
     }


[02/33] incubator-ignite git commit: # ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test

Posted by sb...@apache.org.
# ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test


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

Branch: refs/heads/ignite-950
Commit: 49dc4a50197b8a7203bfc0684d5f77f46bda0297
Parents: 44bbece
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 10:12:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 11:41:00 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |  8 +--
 .../discovery/GridDiscoveryManager.java         | 19 ++++--
 .../continuous/CacheContinuousQueryHandler.java |  8 +++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  4 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  6 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 20 +++---
 .../tcp/TcpCommunicationSpiMBean.java           | 11 ++++
 .../ignite/spi/discovery/DiscoverySpi.java      |  3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 12 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 12 +++-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java | 18 +++++
 ...ridFailFastNodeFailureDetectionSelfTest.java | 17 ++++-
 .../IgniteSlowClientDetectionSelfTest.java      | 69 +++++++++++++++++++-
 .../testframework/GridSpiTestContext.java       |  4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 17 files changed, 180 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 885d52c..40a5ea5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -480,12 +480,12 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.io().messageFactory();
                     }
 
-                    @Override public boolean tryFailNode(UUID nodeId) {
-                        return ctx.discovery().tryFailNode(nodeId);
+                    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
+                        return ctx.discovery().tryFailNode(nodeId, warning);
                     }
 
-                    @Override public void failNode(UUID nodeId) {
-                        ctx.discovery().failNode(nodeId);
+                    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+                        ctx.discovery().failNode(nodeId, warning);
                     }
 
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 717cdf3..1e4b972 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -386,9 +386,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
                 else {
-                    minorTopVer = 0;
+                    if (type != EVT_NODE_SEGMENTED) {
+                        minorTopVer = 0;
 
-                    verChanged = true;
+                        verChanged = true;
+                    }
+                    else
+                        verChanged = false;
                 }
 
                 AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
@@ -1481,15 +1485,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning message to be shown on all nodes.
      * @return Whether node is failed.
      */
-    public boolean tryFailNode(UUID nodeId) {
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return false;
 
         try {
             if (!getSpi().pingNode(nodeId)) {
-                getSpi().failNode(nodeId);
+                getSpi().failNode(nodeId, warning);
 
                 return true;
             }
@@ -1503,13 +1508,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID to fail.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public void failNode(UUID nodeId) {
+    public void failNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return;
 
         try {
-            getSpi().failNode(nodeId);
+            getSpi().failNode(nodeId, warning);
         }
         finally {
             busyLock.leaveBusy();
@@ -1520,6 +1526,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.
+     * @param discoCache Discovery cache.
      * @return {@code True} if topology was updated.
      */
     private boolean updateTopologyVersionIfGreater(AffinityTopologyVersion updated, DiscoCache discoCache) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index ad78b92..ff2905f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.managers.deployment.*;
@@ -226,6 +227,13 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
                             ctx.continuous().addNotification(nodeId, routineId, evt.entry(), topic, sync, true);
                         }
+                        catch (ClusterTopologyCheckedException ex) {
+                            IgniteLogger log = ctx.log(getClass());
+
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send event notification to node, node left cluster " +
+                                    "[node=" + nodeId + ", err=" + ex + ']');
+                        }
                         catch (IgniteCheckedException ex) {
                             U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 18191a1..5e557bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -754,12 +754,12 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public boolean tryFailNode(UUID nodeId) {
+        @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
             return false;
         }
 
         /** {@inheritDoc} */
-        @Override public void failNode(UUID nodeId) {
+        @Override public void failNode(UUID nodeId, @Nullable String warning) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index a655a73..611702b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -307,14 +307,16 @@ public interface IgniteSpiContext {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      * @return If node was failed.
      */
-    public boolean tryFailNode(UUID nodeId);
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param c Timeout object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 538e9a8..84c1a57 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1153,15 +1153,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return msgQueueLimit;
     }
 
-    /**
-     * Gets slow client queue limit.
-     * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
-     * those clients whose queue exceeded this limit.
-     *
-     * @return Slow client queue limit.
-     */
-    public int getSlowClientQueueLimit() {
+    /** {@inheritDoc} */
+    @Override public int getSlowClientQueueLimit() {
         return slowClientQueueLimit;
     }
 
@@ -1923,10 +1916,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
+
+                    LT.warn(log, null, msg);
 
-                    getSpiContext().failNode(id);
+                    getSpiContext().failNode(id, msg);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index fe4f581..1971d99 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -268,4 +268,15 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Maximum number of unacknowledged messages.")
     public int getUnacknowledgedMessagesBufferSize();
+
+    /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    @MXBeanDescription("Slow client queue limit.")
+    public int getSlowClientQueueLimit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index b952087..11a18b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -151,8 +151,9 @@ public interface DiscoverySpi extends IgniteSpi {
      * Initiates failure of provided node.
      *
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * Whether or not discovery is started in client mode.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index fef6f4f..e255e08 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -335,13 +335,15 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = rmtNodes.get(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -1432,6 +1434,14 @@ class ClientImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
+                if (msg.warning() != null) {
+                    ClusterNode creatorNode = rmtNodes.get(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 8eb82ac..2458f85 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -584,13 +584,15 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = ring.node(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -3409,6 +3411,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         worker.interrupt();
                 }
 
+                if (msg.warning() != null && !msg.creatorNodeId().equals(getLocalNodeId())) {
+                    ClusterNode creatorNode = ring.node(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 94097c9..ace917f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -175,8 +175,9 @@ abstract class TcpDiscoveryImpl {
 
     /**
      * @param nodeId Node id.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public abstract void failNode(UUID nodeId);
+    public abstract void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param gridName Grid name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index c36ac76..1d1916a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -373,8 +373,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        impl.failNode(nodeId);
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+        impl.failNode(nodeId, warning);
     }
 
     /** {@inheritDoc} */
@@ -385,7 +385,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** {@inheritDoc} */
     @Override public boolean isClientMode() {
         if (impl == null)
-            throw new IllegalStateException("TcpDiscoverySpi has not started");
+            throw new IllegalStateException("TcpDiscoverySpi has not started.");
 
         return impl instanceof ClientImpl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
index 8cb8414..93ecdaa 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -37,6 +38,9 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     /** Internal order of the failed node. */
     private final long order;
 
+    /** */
+    private String warning;
+
     /**
      * Constructor.
      *
@@ -55,6 +59,20 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
+     * @param warning Warning message to be shown on all nodes.
+     */
+    public void warning(String warning) {
+        this.warning = warning;
+    }
+
+    /**
+     * @return Warning message to be shown on all nodes.
+     */
+    @Nullable public String warning() {
+        return warning;
+    }
+
+    /**
      * Gets ID of the failed node.
      *
      * @return ID of the failed node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 992d7bf..238115d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -50,7 +50,12 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(IP_FINDER);
-        disco.setHeartbeatFrequency(10000);
+        disco.setHeartbeatFrequency(10_000);
+
+        // Set parameters for fast ping failure.
+        disco.setSocketTimeout(100);
+        disco.setNetworkTimeout(100);
+        disco.setReconnectCount(2);
 
         cfg.setDiscoverySpi(disco);
 
@@ -66,8 +71,6 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testFailFast() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-933");
-
         startGridsMultiThreaded(5);
 
         final CountDownLatch failLatch = new CountDownLatch(4);
@@ -87,6 +90,8 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         Ignite ignite1 = ignite(0);
         Ignite ignite2 = ignite(1);
 
+        final CountDownLatch evtLatch = new CountDownLatch(1);
+
         ignite1.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
                 respond(rcvMsg);
@@ -95,12 +100,18 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         ignite2.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
+                evtLatch.countDown();
+
                 respond(rcvMsg);
             }
         });
 
         ignite1.message(ignite1.cluster().forRemotes()).send(null, "Message");
 
+        evtLatch.await(); // Wait when connection is established.
+
+        log.info("Fail node: " + ignite1.cluster().localNode());
+
         failNode(ignite1);
 
         assert failLatch.await(1000, MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 09b4215..27c2a61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -19,24 +19,37 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.event.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
 
 /**
  *
  */
 public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
-
+    /** */
     public static final String PARTITIONED = "partitioned";
 
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @return Node count.
      */
@@ -48,6 +61,8 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
         if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
             cfg.setClientMode(true);
 
@@ -66,7 +81,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(nodeCount());
+        startGrids(nodeCount());
     }
 
     /** {@inheritDoc} */
@@ -82,6 +97,45 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     public void testSlowClient() throws Exception {
         final IgniteEx slowClient = grid(nodeCount() - 1);
 
+        final ClusterNode slowClientNode = slowClient.localNode();
+
+        final CountDownLatch evtSegmentedLatch = new CountDownLatch(1);
+
+        slowClient.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_SEGMENTED);
+
+                DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+                assertEquals(slowClientNode, evt0.eventNode());
+                assertEquals(5L, evt0.topologyVersion());
+
+                evtSegmentedLatch.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        final CountDownLatch evtFailedLatch = new CountDownLatch(nodeCount() - 1);
+
+        for (int i = 0; i < nodeCount() - 1; i++) {
+            grid(i).events().localListen(new IgnitePredicate<Event>() {
+                @Override public boolean apply(Event evt) {
+                    assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_FAILED);
+
+                    DiscoveryEvent evt0 = (DiscoveryEvent) evt;
+
+                    assertEquals(slowClientNode, evt0.eventNode());
+                    assertEquals(6L, evt0.topologyVersion());
+                    assertEquals(4, evt0.topologyNodes().size());
+
+                    evtFailedLatch.countDown();
+
+                    return false;
+                }
+            }, EventType.EVT_NODE_FAILED);
+        }
+
         assertTrue(slowClient.cluster().localNode().isClient());
 
         IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
@@ -109,14 +163,23 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < 100; i++)
             cache0.put(0, new byte[10 * 1024]);
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
             }
         }, getTestTimeout());
+
+        assertTrue(wait);
+
+        assertTrue("Failed to wait for client failed event", evtFailedLatch.await(5000, MILLISECONDS));
+        assertTrue("Failed to wait for client segmented event", evtSegmentedLatch.await(5000, MILLISECONDS));
     }
 
+    /**
+     *
+     */
     private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        /** {@inheritDoc} */
         @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
             System.out.println(">>>> Received update: " + iterable);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index c20ff2e..08268af 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -497,12 +497,12 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tryFailNode(UUID nodeId) {
+    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index b4977ce..2d14728 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -91,6 +91,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridMessageListenSelfTest.class);
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
+        suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
 
         return suite;
     }


[16/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973


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

Branch: refs/heads/ignite-950
Commit: de27ff1b8788b622937b17bfeaf8941242914d1a
Parents: bcfbb92 415264e
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:21:46 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:21:46 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  71 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 21 files changed, 467 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[27/33] incubator-ignite git commit: #IGNITE Added message on init stop.

Posted by sb...@apache.org.
#IGNITE Added message on init stop.


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

Branch: refs/heads/ignite-950
Commit: 16059962e303f7580c6524b1e0f93c2dcce9eaa4
Parents: de0930d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 15:30:48 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 15:30:48 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 3 files changed, 130 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 48f24ac..9f7c983 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -124,10 +124,18 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
 
     /** {@inheritDoc} */
     @Override protected String className(int id) throws IgniteCheckedException {
-        if (cache == null)
+        GridCacheAdapter<Integer, String> cache0 = cache;
+
+        if (cache0 == null) {
             U.awaitQuiet(latch);
 
-        String clsName = cache.get(id);
+            cache0 = cache;
+
+            if (cache0 == null)
+                throw new IllegalStateException("Failed to initialize marshaller context (grid is stopping).");
+        }
+
+        String clsName = cache0.get(id);
 
         if (clsName == null) {
             File file = new File(workDir, id + ".classname");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
new file mode 100644
index 0000000..c56ad1c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test cache operations with daemon node.
+ */
+public class GridCacheDaemonNodeStopSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Daemon flag. */
+    protected boolean daemon;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        daemon = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        c.setDaemon(daemon);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        c.setConnectorConfiguration(null);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(LOCAL);
+        cc.setAtomicityMode(ATOMIC);
+
+        TransactionConfiguration cfg = new TransactionConfiguration();
+
+        c.setTransactionConfiguration(cfg);
+
+        c.setCacheConfiguration(cc);
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop() throws Exception {
+        try {
+            daemon = true;
+
+            final IgniteEx daemonNode = startGrid(0);
+
+            final IgniteInternalFuture<Object> f = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    daemonNode.context().marshallerContext().getClass(1,
+                        GridCacheDaemonNodeStopSelfTest.class.getClassLoader());
+
+                    return null;
+                }
+            });
+
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    f.get(300);
+
+                    return null;
+                }
+            }, IgniteFutureTimeoutCheckedException.class);
+
+            // Stop grid.
+            stopGrid(0);
+
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    f.get(5, TimeUnit.SECONDS);
+
+                    return null;
+                }
+            }, IllegalStateException.class);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index 5947d33..b83f50f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -100,6 +100,7 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTestSuite(GridCacheDaemonNodeLocalSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodePartitionedSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodeReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheDaemonNodeStopSelfTest.class);
 
         // Write-behind.
         suite.addTest(IgniteCacheWriteBehindTestSuite.suite());


[10/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1034' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1034' into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: 7a8b572bf93b2d72aeb19cdad1f904fed49d5d17
Parents: d699faa d34f329
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 17:27:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 17:27:55 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  66 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 18 files changed, 451 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[22/33] incubator-ignite git commit: #IGNITE-GG-10449 marshal credentials in client discovery.

Posted by sb...@apache.org.
#IGNITE-GG-10449 marshal credentials in client discovery.


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

Branch: refs/heads/ignite-950
Commit: a3d8e603a5c9a7fbf3f81a7dd2aded2a8139f578
Parents: 1cc0000
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 11:33:53 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 11:33:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 23 ++++++++++++++++++++
 1 file changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a3d8e603/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index e255e08..68e5dbd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -362,6 +362,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         long startTime = U.currentTimeMillis();
 
+        // Marshal credentials for backward compatibility and security.
+        marshalCredentials(locNode);
+
         while (true) {
             if (Thread.currentThread().isInterrupted())
                 throw new InterruptedException();
@@ -541,6 +544,26 @@ class ClientImpl extends TcpDiscoveryImpl {
         return null;
     }
 
+    /**
+     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+     *
+     * @param node Node to marshall credentials for.
+     * @throws IgniteSpiException If marshalling failed.
+     */
+    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            // Use security-unsafe getter.
+            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
+                spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+
+            node.setAttributes(attrs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+        }
+    }
 
     /**
      * @param topVer New topology version.


[33/33] incubator-ignite git commit: ignite-950: implementing marshal aware

Posted by sb...@apache.org.
ignite-950: implementing marshal aware


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

Branch: refs/heads/ignite-950
Commit: 5749b068d623072299b46eb1fb099c8e3636d5ee
Parents: d9f85e6
Author: Denis Magda <dm...@gridgain.com>
Authored: Tue Jun 23 16:51:33 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Tue Jun 23 16:51:33 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  |   1 -
 .../cache/CacheIndexedObjectImpl.java           |   2 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   3 +-
 .../optimized/OptimizedClassDescriptor.java     |  38 +-
 .../optimized/OptimizedFieldsReader.java        | 196 ++++++++++
 .../optimized/OptimizedFieldsWriter.java        | 196 ++++++++++
 .../optimized/OptimizedMarshalAware.java        |  52 +++
 .../OptimizedMarshalAwareMetaCollector.java     | 179 +++++++++
 .../optimized/OptimizedMarshallerExt.java       | 357 +++++++++++++++++
 .../OptimizedMarshallerMetaHandler.java         |  40 ++
 .../optimized/OptimizedMarshallerUtils.java     |   3 +
 .../optimized/OptimizedObjectInputStream.java   | 392 ++++++++++++++++++-
 .../OptimizedObjectInputStreamExt.java          |  51 +++
 .../optimized/OptimizedObjectMetadata.java      | 172 ++++++++
 .../optimized/OptimizedObjectMetadataKey.java   |  70 ++++
 .../optimized/OptimizedObjectOutputStream.java  | 209 +++++++++-
 .../OptimizedObjectOutputStreamExt.java         | 152 +++++++
 .../OptimizedObjectStreamExtRegistry.java       | 225 +++++++++++
 .../optimized/ext/OptimizedMarshallerExt.java   | 353 -----------------
 .../ext/OptimizedMarshallerExtMetaHandler.java  |  40 --
 .../ext/OptimizedObjectInputStreamExt.java      | 241 ------------
 .../optimized/ext/OptimizedObjectMetadata.java  | 157 --------
 .../ext/OptimizedObjectMetadataKey.java         |  70 ----
 .../ext/OptimizedObjectOutputStreamExt.java     | 179 ---------
 .../ext/OptimizedObjectStreamExtRegistry.java   | 226 -----------
 .../marshaller/optimized/ext/package-info.java  |  21 -
 .../ext/OptimizedMarshallerExtSelfTest.java     |  75 +++-
 .../junits/IgniteTestResources.java             |   1 -
 ...acheOptimizedMarshallerExtQuerySelfTest.java |   4 +-
 30 files changed, 2405 insertions(+), 1302 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 422f517..7356d85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -32,7 +32,6 @@ import org.apache.ignite.logger.java.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.plugin.segmentation.*;
 import org.apache.ignite.resources.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
index 58e9c97..47b83ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheIndexedObjectImpl.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.jetbrains.annotations.*;
 import sun.misc.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index 42ccbce..a4964b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -22,7 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
+import org.apache.ignite.marshaller.optimized.*;
 import org.jetbrains.annotations.*;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index 6ae958a..d29b64a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.marshaller.optimized.ext.*;
 import org.jetbrains.annotations.*;
 
 import java.math.*;
@@ -90,7 +89,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
         if (marsh instanceof OptimizedMarshallerExt) {
             optMarshExt = (OptimizedMarshallerExt)marsh;
 
-            OptimizedMarshallerExtMetaHandler metaHandler = new OptimizedMarshallerExtMetaHandler() {
+            OptimizedMarshallerMetaHandler metaHandler = new OptimizedMarshallerMetaHandler() {
                 @Override public void addMeta(int typeId, OptimizedObjectMetadata meta) {
                     if (metaBuf.contains(typeId))
                         return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
index d77551f..126568e 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedClassDescriptor.java
@@ -329,6 +329,21 @@ public class OptimizedClassDescriptor {
                         throw new IOException("Externalizable class doesn't have default constructor: " + cls, e);
                     }
                 }
+                else if (OptimizedMarshalAware.class.isAssignableFrom(cls)) {
+                    type = MARSHAL_AWARE;
+
+                    try {
+                        constructor = !Modifier.isStatic(cls.getModifiers()) && cls.getDeclaringClass() != null ?
+                            cls.getDeclaredConstructor(cls.getDeclaringClass()) :
+                            cls.getDeclaredConstructor();
+
+                        constructor.setAccessible(true);
+                    }
+                    catch (NoSuchMethodException e) {
+                        throw new IOException("OptimizedMarshalAware class doesn't have default constructor: " + cls,
+                            e);
+                    }
+                }
                 else {
                     type = SERIALIZABLE;
 
@@ -408,8 +423,6 @@ public class OptimizedClassDescriptor {
                                 isPrivate(mod) && isStatic(mod) && isFinal(mod)) {
                                 hasSerialPersistentFields = true;
 
-                                fieldsIndexingSupported = false;
-
                                 serFieldsDesc.setAccessible(true);
 
                                 ObjectStreamField[] serFields = (ObjectStreamField[]) serFieldsDesc.get(null);
@@ -718,6 +731,22 @@ public class OptimizedClassDescriptor {
 
                 break;
 
+            case MARSHAL_AWARE:
+                writeTypeData(out);
+
+                out.writeShort(checksum);
+                out.writeMarshalAware(obj);
+
+                if (out.metaHandler.metadata(typeId) == null) {
+                    OptimizedMarshalAwareMetaCollector collector = new OptimizedMarshalAwareMetaCollector();
+
+                    ((OptimizedMarshalAware)obj).writeFields(collector);
+
+                    out.metaHandler.addMeta(typeId, collector.meta());
+                }
+
+                break;
+
             case SERIALIZABLE:
                 if (out.requireSerializable() && !isSerial)
                     throw new NotSerializableException("Must implement java.io.Serializable or " +
@@ -770,6 +799,11 @@ public class OptimizedClassDescriptor {
 
                 return in.readSerializable(cls, readObjMtds, readResolveMtd, fields);
 
+            case MARSHAL_AWARE:
+                verifyChecksum(in.readShort());
+
+                return in.readMarshalAware(constructor, readResolveMtd);
+
             default:
                 assert false : "Unexpected type: " + type;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
new file mode 100644
index 0000000..57a59ed
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsReader.java
@@ -0,0 +1,196 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public interface OptimizedFieldsReader {
+    /**
+     * @param fieldName Field name.
+     * @return Byte value.
+     * @throws IOException In case of error.
+     */
+    public byte readByte(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short value.
+     * @throws IOException In case of error.
+     */
+    public short readShort(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer value.
+     * @throws IOException In case of error.
+     */
+    public int readInt(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long value.
+     * @throws IOException In case of error.
+     */
+    public long readLong(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @throws IOException In case of error.
+     * @return Float value.
+     */
+    public float readFloat(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Double value.
+     * @throws IOException In case of error.
+     */
+    public double readDouble(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char value.
+     * @throws IOException In case of error.
+     */
+    public char readChar(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean value.
+     * @throws IOException In case of error.
+     */
+    public boolean readBoolean(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public String readString(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T> T readObject(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public byte[] readByteArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Short array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public short[] readShortArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Integer array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public int[] readIntArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Long array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public long[] readLongArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Float array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public float[] readFloatArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Byte array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public double[] readDoubleArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Char array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public char[] readCharArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Boolean array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public boolean[] readBooleanArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return String array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public String[] readStringArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Object array.
+     * @throws IOException In case of error.
+     */
+    @Nullable public Object[] readObjectArray(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Collection.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T> Collection<T> readCollection(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Map.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <K, V> Map<K, V> readMap(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T readEnum(String fieldName) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @return Value.
+     * @throws IOException In case of error.
+     */
+    @Nullable public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
new file mode 100644
index 0000000..f104fba
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedFieldsWriter.java
@@ -0,0 +1,196 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+public interface OptimizedFieldsWriter {
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeByte(String fieldName, byte val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeShort(String fieldName, short val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeInt(String fieldName, int val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeLong(String fieldName, long val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeFloat(String fieldName, float val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeDouble(String fieldName, double val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeChar(String fieldName, char val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeBoolean(String fieldName, boolean val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeString(String fieldName, @Nullable String val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param obj Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeObject(String fieldName, @Nullable Object obj) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param col Collection to write.
+     * @throws IOException In case of error.
+     */
+    public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param map Map to write.
+     * @throws IOException In case of error.
+     */
+    public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException;
+
+    /**
+     * @param fieldName Field name.
+     * @param val Value to write.
+     * @throws IOException In case of error.
+     */
+    public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
new file mode 100644
index 0000000..cc90cff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAware.java
@@ -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.
+ */
+
+package org.apache.ignite.marshaller.optimized;
+
+import java.io.*;
+
+/**
+ * TODO: IGNITE-950
+ *
+ * Interface that allows to implement custom serialization
+ * logic for portable objects. Portable objects are not required
+ * to implement this interface, in which case Ignite will automatically
+ * serialize portable objects using reflection.
+ * <p>
+ * This interface, in a way, is analogous to {@link java.io.Externalizable}
+ * interface, which allows users to override default serialization logic,
+ * usually for performance reasons. The only difference here is that portable
+ * serialization is already very fast and implementing custom serialization
+ * logic for portables does not provide significant performance gains.
+ */
+public interface OptimizedMarshalAware {
+    /**
+     * Writes fields to provided writer.
+     *
+     * @param writer Fields writer.
+     * @throws IOException In case of error.
+     */
+    public void writeFields(OptimizedFieldsWriter writer) throws IOException;
+
+    /**
+     * Reads fields from provided reader.
+     *
+     * @param reader Fields reader.
+     * @throws IOException In case of error.
+     */
+    public void readFields(OptimizedFieldsReader reader) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java
new file mode 100644
index 0000000..c4fb9d4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshalAwareMetaCollector.java
@@ -0,0 +1,179 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedFieldType.*;
+
+/**
+ * TODO: IGNITE-950
+ */
+class OptimizedMarshalAwareMetaCollector implements OptimizedFieldsWriter {
+    /** */
+    private OptimizedObjectMetadata meta;
+
+    /**
+     * Constructor.
+     */
+    public OptimizedMarshalAwareMetaCollector() {
+        meta = new OptimizedObjectMetadata();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws IOException {
+        putFieldToMeta(fieldName, BYTE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws IOException {
+        putFieldToMeta(fieldName, SHORT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws IOException {
+        putFieldToMeta(fieldName, INT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws IOException {
+        putFieldToMeta(fieldName, LONG);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws IOException {
+        putFieldToMeta(fieldName, FLOAT);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws IOException {
+        putFieldToMeta(fieldName, DOUBLE);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws IOException {
+        putFieldToMeta(fieldName, CHAR);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws IOException {
+        putFieldToMeta(fieldName, BOOLEAN);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException {
+        putFieldToMeta(fieldName, OTHER);
+    }
+
+    /**
+     * Returns gather metadata.
+     *
+     * @return Metadata.
+     */
+    OptimizedObjectMetadata meta() {
+        return meta;
+    }
+
+    /**
+     * Adds field to the metadata.
+     *
+     * @param fieldName Field name.
+     * @param type Field type.
+     */
+    private void putFieldToMeta(String fieldName, OptimizedFieldType type) {
+        meta.addMeta(OptimizedMarshallerUtils.resolveFieldId(fieldName), type);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
new file mode 100644
index 0000000..7d90581
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerExt.java
@@ -0,0 +1,357 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedClassDescriptor.*;
+
+/**
+ * TODO
+ */
+public class OptimizedMarshallerExt extends OptimizedMarshaller {
+    /** */
+    static final byte EMPTY_FOOTER = -1;
+
+    /** */
+    static final byte FOOTER_LEN_OFF = 2;
+
+    /** */
+    static final int FOOTER_BODY_LEN_MASK = 0x3FFFFFFF;
+
+    /** */
+    static final int FOOTER_BODY_IS_HANDLE_MASK = 0x40000000;
+
+    /** */
+    static final byte FOOTER_BODY_HANDLE_MASK_BIT = 30;
+
+    /** */
+    public static final byte VARIABLE_LEN = -1;
+
+    /** */
+    private volatile OptimizedMarshallerMetaHandler metaHandler;
+
+    /**
+     * Creates new marshaller will all defaults.
+     *
+     * @throws IgniteException If this marshaller is not supported on the current JVM.
+     */
+    public OptimizedMarshallerExt() {
+        // No-op
+    }
+
+    /**
+     * Creates new marshaller providing whether it should
+     * require {@link Serializable} interface or not.
+     *
+     * @param requireSer Whether to require {@link Serializable}.
+     */
+    public OptimizedMarshallerExt(boolean requireSer) {
+        super(requireSer);
+    }
+
+    /**
+     * Sets metadata handler.
+     *
+     * @param metaHandler Metadata handler.
+     */
+    public void setMetadataHandler(OptimizedMarshallerMetaHandler metaHandler) {
+        this.metaHandler = metaHandler;
+    }
+
+    /**
+     * Returns currently set ID mapper.
+     *
+     * @return ID mapper.
+     */
+    public OptimizedMarshallerIdMapper idMapper() {
+        return mapper;
+    }
+
+    /**
+     * Enables fields indexing for the object of the given {@code cls}.
+     *
+     * If enabled then a footer will be added during marshalling of an object of the given {@code cls} to the end of
+     * its serialized form.
+     *
+     * @param cls Class.
+     * @return {@code true} if fields indexing is enabled.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public boolean enableFieldsIndexing(Class<?> cls) throws IgniteCheckedException {
+        assert metaHandler != null;
+
+        if (ctx.isSystemType(cls.getName()))
+            return false;
+
+        if (OptimizedMarshalAware.class.isAssignableFrom(cls))
+            return true;
+
+        try {
+            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
+
+            if (desc.fields() != null && desc.fields().fieldsIndexingSupported()) {
+                //The function is called on kernel startup, calling metaHandler.metadata() will hang the grid,
+                //because the underlying cache is not ready.
+                //if (metaHandler.metadata(desc.typeId()) != null)
+                //    return true;
+
+                OptimizedObjectMetadata meta = new OptimizedObjectMetadata();
+
+                for (ClassFields clsFields : desc.fields().fieldsList())
+                    for (FieldInfo info : clsFields.fieldInfoList())
+                        meta.addMeta(info.id(), info.type());
+
+                metaHandler.addMeta(desc.typeId(), meta);
+
+                return true;
+            }
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to put meta for class: " + cls.getName(), e);
+        }
+
+        return false;
+    }
+
+    /**
+     * Checks whether fields indexing is enabled for objects of the given {@code cls}.
+     *
+     * @param cls Class.
+     * @return {@code true} if fields indexing is enabled.
+     */
+    public boolean fieldsIndexingEnabled(Class<?> cls) {
+        assert metaHandler != null;
+
+        if (ctx.isSystemType(cls.getName()))
+            return false;
+
+        if (OptimizedMarshalAware.class.isAssignableFrom(cls))
+            return true;
+
+        try {
+            OptimizedClassDescriptor desc = OptimizedMarshallerUtils.classDescriptor(clsMap, cls, ctx, mapper);
+
+            return desc.fields() != null && desc.fields().fieldsIndexingSupported() &&
+                metaHandler.metadata(desc.typeId()) != null;
+        }
+        catch (IOException e) {
+            throw new IgniteException("Failed to load class description: " + cls);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setPoolSize(int poolSize) {
+        OptimizedObjectStreamExtRegistry.poolSize(poolSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void marshal(@Nullable Object obj, OutputStream out) throws IgniteCheckedException {
+        assert out != null;
+
+        OptimizedObjectOutputStreamExt objOut = null;
+
+        try {
+            objOut = OptimizedObjectStreamExtRegistry.out();
+
+            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
+
+            objOut.out().outputStream(out);
+
+            objOut.writeObject(obj);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeOut(objOut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte[] marshal(@Nullable Object obj) throws IgniteCheckedException {
+        OptimizedObjectOutputStreamExt objOut = null;
+
+        try {
+            objOut = OptimizedObjectStreamExtRegistry.out();
+
+            objOut.context(clsMap, ctx, mapper, requireSer, metaHandler);
+
+            objOut.writeObject(obj);
+
+            return objOut.out().array();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to serialize object: " + obj, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeOut(objOut);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unmarshal(InputStream in, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        assert in != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().inputStream(in);
+
+            return (T)objIn.readObject();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                             "(make sure same versions of all classes are available on all nodes or " +
+                                             "enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unmarshal(byte[] arr, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        return unmarshal(arr, 0, arr.length, clsLdr);
+    }
+
+    /**
+     * Unmarshals object from byte array using given class loader and offset with len.
+     *
+     * @param <T> Type of unmarshalled object.
+     * @param arr Byte array.
+     * @param off Object's offset in the array.
+     * @param len Object's length in the array.
+     * @param clsLdr Class loader to use.
+     * @return Unmarshalled object.
+     * @throws IgniteCheckedException If unmarshalling failed.
+     */
+     public <T> T unmarshal(byte[] arr, int off, int len, @Nullable ClassLoader clsLdr) throws IgniteCheckedException {
+        assert arr != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return (T)objIn.readObject();
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                                 "(make sure same version of all classes are available on all nodes or" +
+                                                 " enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /**
+     * Checks whether object, serialized to byte array {@code arr}, has a field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @param arr Object's serialized form.
+     * @param off Object's start off.
+     * @param len Object's len.
+     * @return {@code true} if field exists.
+     */
+    public boolean hasField(String fieldName, byte[] arr, int off, int len) throws IgniteCheckedException {
+        assert arr != null && fieldName != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return objIn.hasField(fieldName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+
+    /**
+     * Looks up field with the given name and returns it in one of the following representations. If the field is
+     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
+     * for future processing. In all other cases the field is fully deserialized.
+     *
+     * @param fieldName Field name.
+     * @param arr Object's serialized form.
+     * @param off Object's start offset.
+     * @param len Object's len.
+     * @param clsLdr Class loader.
+     * @param <T> Expected field class.
+     * @return Field.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public <T> T readField(String fieldName, byte[] arr, int off, int len, @Nullable ClassLoader clsLdr)
+        throws IgniteCheckedException {
+
+        assert arr != null && fieldName != null;
+
+        OptimizedObjectInputStreamExt objIn = null;
+
+        try {
+            objIn = OptimizedObjectStreamExtRegistry.in();
+
+            objIn.context(clsMap, ctx, mapper, clsLdr != null ? clsLdr : dfltClsLdr, metaHandler);
+
+            objIn.in().bytes(arr, off, len);
+
+            return objIn.readField(fieldName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to find field with name: " + fieldName, e);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
+                                             "(make sure same version of all classes are available on all nodes or" +
+                                             " enable peer-class-loading): " + clsLdr, e);
+        }
+        finally {
+            OptimizedObjectStreamExtRegistry.closeIn(objIn);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
new file mode 100644
index 0000000..5fad57c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerMetaHandler.java
@@ -0,0 +1,40 @@
+/*
+ * 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.marshaller.optimized;
+
+/**
+ * Metadata handler for optimized objects.
+ */
+public interface OptimizedMarshallerMetaHandler {
+    /**
+     * Adds meta data.
+     *
+     * @param typeId Type ID.
+     * @param meta Meta data.
+     */
+    void addMeta(int typeId, OptimizedObjectMetadata meta);
+
+
+    /**
+     * Gets meta data for provided type ID.
+     *
+     * @param typeId Type ID.
+     * @return Meta data.
+     */
+    OptimizedObjectMetadata metadata(int typeId);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
index 7a7ee69..cd25f1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerUtils.java
@@ -144,6 +144,9 @@ public class OptimizedMarshallerUtils {
     /** */
     public static final byte SERIALIZABLE = 102;
 
+    /** */
+    public static final byte MARSHAL_AWARE = 103;
+
     /** UTF-8 character name. */
     static final Charset UTF_8 = Charset.forName("UTF-8");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
index c8a85b2..6eeadc8 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStream.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.marshaller.optimized;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
+import org.jetbrains.annotations.*;
 import sun.misc.*;
 
 import java.io.*;
@@ -30,11 +32,12 @@ import java.util.*;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerExt.*;
 
 /**
  * Optimized object input stream.
  */
-public class OptimizedObjectInputStream extends ObjectInputStream {
+public class OptimizedObjectInputStream extends ObjectInputStream implements OptimizedFieldsReader {
     /** Unsafe. */
     private static final Unsafe UNSAFE = GridUnsafe.unsafe();
 
@@ -51,6 +54,9 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     protected ClassLoader clsLdr;
 
     /** */
+    protected OptimizedMarshallerMetaHandler metaHandler;
+
+    /** */
     protected GridDataInput in;
 
     /** */
@@ -95,6 +101,25 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     }
 
     /**
+     * @param clsMap Class descriptors by class map.
+     * @param ctx Context.
+     * @param mapper ID mapper.
+     * @param clsLdr Class loader.
+     * @param metaHandler Metadata handler.
+     */
+    protected void context(
+        ConcurrentMap<Class, OptimizedClassDescriptor> clsMap,
+        MarshallerContext ctx,
+        OptimizedMarshallerIdMapper mapper,
+        ClassLoader clsLdr,
+        OptimizedMarshallerMetaHandler metaHandler)
+    {
+        context(clsMap, ctx, mapper, clsLdr);
+
+        this.metaHandler = metaHandler;
+    }
+
+    /**
      * @return Input.
      */
     public GridDataInput in() {
@@ -115,6 +140,7 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         ctx = null;
         clsLdr = null;
         clsMap = null;
+        metaHandler = null;
     }
 
     /** {@inheritDoc} */
@@ -241,6 +267,7 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
             case ENUM:
             case EXTERNALIZABLE:
             case SERIALIZABLE:
+            case MARSHAL_AWARE:
                 int typeId = readInt();
 
                 OptimizedClassDescriptor desc = typeId == 0 ?
@@ -481,6 +508,46 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
     }
 
     /**
+     * Reads {@link OptimizedMarshalAware} object.
+     *
+     * @param constructor Constructor.
+     * @param readResolveMtd {@code readResolve} method.
+     * @return Object.
+     * @throws ClassNotFoundException If class not found.
+     * @throws IOException In case of error.
+     */
+    Object readMarshalAware(Constructor<?> constructor, Method readResolveMtd)
+        throws ClassNotFoundException, IOException {
+        Object obj;
+
+        try {
+            obj = constructor.newInstance();
+        }
+        catch (InstantiationException | InvocationTargetException | IllegalAccessException e) {
+            throw new IOException(e);
+        }
+
+        int handle = handles.assign(obj);
+
+        OptimizedMarshalAware extObj = ((OptimizedMarshalAware)obj);
+
+        extObj.readFields(this);
+
+        if (readResolveMtd != null) {
+            try {
+                obj = readResolveMtd.invoke(obj);
+
+                handles.set(handle, obj);
+            }
+            catch (IllegalAccessException | InvocationTargetException e) {
+                throw new IOException(e);
+            }
+        }
+
+        return obj;
+    }
+
+    /**
      * Reads serializable object.
      *
      * @param cls Class.
@@ -935,6 +1002,126 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         return new GetFieldImpl(this);
     }
 
+    /** {@inheritDoc} */
+    @Override public byte readByte(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String readString(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> T readObject(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public byte[] readByteArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public short[] readShortArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public int[] readIntArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public long[] readLongArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public float[] readFloatArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public double[] readDoubleArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public char[] readCharArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public boolean[] readBooleanArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String[] readStringArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public Object[] readObjectArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T> Collection<T> readCollection(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <K, V> Map<K, V> readMap(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T readEnum(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public <T extends Enum<?>> T[] readEnumArray(String fieldName) throws IOException {
+        return doReadField(fieldName);
+    }
+
     /**
      * Skips object footer from the underlying stream.
      *
@@ -955,6 +1142,209 @@ public class OptimizedObjectInputStream extends ObjectInputStream {
         return 0;
     }
 
+    /**
+     * Checks whether the object has a field with name {@code fieldName}.
+     *
+     * @param fieldName Field name.
+     * @return {@code true} if field exists, {@code false} otherwise.
+     * @throws IOException in case of error.
+     */
+    public boolean hasField(String fieldName) throws IOException {
+        int pos = in.position();
+
+        byte type = in.readByte();
+
+        if (type != SERIALIZABLE && type != MARSHAL_AWARE) {
+            in.position(pos);
+            return false;
+        }
+
+        FieldRange range = fieldRange(fieldName, pos);
+
+        in.position(pos);
+
+        return range != null && range.start > 0;
+    }
+
+    /**
+     * Looks up field with the given name and returns it in one of the following representations. If the field is
+     * serializable and has a footer then it's not deserialized but rather returned wrapped by {@link CacheObjectImpl}
+     * for future processing. In all other cases the field is fully deserialized.
+     *
+     * @param fieldName Field name.
+     * @return Field.
+     * @throws IOException In case of error.
+     * @throws ClassNotFoundException In case of error.
+     */
+    public <F> F readField(String fieldName) throws IOException, ClassNotFoundException {
+        return doReadField(fieldName, false);
+    }
+
+    /**
+     * Reads the field using footer.
+     *
+     * @param fieldName Field name.
+     * @param deserialize Deserialize field if it supports footer.
+     * @param <F> Field type.
+     * @return Field.
+     */
+    private <F> F doReadField(String fieldName, boolean deserialize) throws IOException, ClassNotFoundException {
+        int pos = in.position();
+
+        byte type = in.readByte();
+
+        if (type != SERIALIZABLE && type != MARSHAL_AWARE) {
+            in.position(pos);
+            return null;
+        }
+
+        FieldRange range = fieldRange(fieldName, pos);
+
+        F field = null;
+
+        if (range != null && range.start >= 0) {
+            in.position(range.start);
+
+            if (deserialize)
+                field = (F)readObject();
+            else {
+                byte fieldType = in.readByte();
+
+                if ((fieldType == SERIALIZABLE && metaHandler.metadata(in.readInt()) != null) ||
+                    fieldType == MARSHAL_AWARE)
+                    //Do we need to make a copy of array?
+                    field = (F)new CacheIndexedObjectImpl(in.array(), range.start, range.len);
+                else {
+                    in.position(range.start);
+                    field = (F)readObject();
+                }
+            }
+        }
+
+        in.position(pos);
+
+        return field;
+    }
+
+    /**
+     * Reads the field and deserializes it.
+     *
+     * @param fieldName Field name.
+     * @param <F> Field type.
+     * @return Field.
+     * @throws IOException In case of error.
+     */
+    private <F> F doReadField(String fieldName) throws IOException {
+        try {
+            return doReadField(fieldName, true);
+        }
+        catch (ClassNotFoundException e) {
+            throw new IOException("Failed to read the field, class definition has not" +
+                " been found [field=" + fieldName + "]");
+        }
+    }
+
+    /**
+     * Returns field offset in the byte stream.
+     *
+     * @param fieldName Field name.
+     * @param start Object's start offset.
+     * @return positive range or {@code null} if the object doesn't have such a field.
+     * @throws IOException in case of error.
+     */
+    private FieldRange fieldRange(String fieldName, int start) throws IOException {
+        int fieldId = resolveFieldId(fieldName);
+
+        int typeId = readInt();
+
+        int clsNameLen = 0;
+
+        if (typeId == 0) {
+            int pos = in.position();
+
+            typeId = OptimizedMarshallerUtils.resolveTypeId(readUTF(), mapper);
+
+            clsNameLen = in.position() - pos;
+        }
+
+        OptimizedObjectMetadata meta = metaHandler.metadata(typeId);
+
+        if (meta == null)
+            // TODO: IGNITE-950 add warning!
+            return null;
+
+        int end = in.size();
+
+        in.position(end - FOOTER_LEN_OFF);
+
+        short footerLen = in.readShort();
+
+        if (footerLen == EMPTY_FOOTER)
+            return null;
+
+        // +2 - skipping length at the beginning
+        int footerOff = (end - footerLen) + 2;
+        in.position(footerOff);
+
+        int fieldOff = 0;
+
+        for (OptimizedObjectMetadata.FieldInfo info : meta.getMeta()) {
+            int len;
+            boolean isHandle;
+
+            if (info.length() == VARIABLE_LEN) {
+                int fieldInfo = in.readInt();
+
+                len = fieldInfo & FOOTER_BODY_LEN_MASK;
+                isHandle = ((fieldInfo & FOOTER_BODY_IS_HANDLE_MASK) >> FOOTER_BODY_HANDLE_MASK_BIT) == 1;
+            }
+            else {
+                len = info.length();
+                isHandle = false;
+            }
+
+            if (info.id() == fieldId) {
+                if (!isHandle) {
+                    //object header len: 1 - for type, 4 - for type ID, 2 - for checksum.
+                    fieldOff += 1 + 4 + clsNameLen + 2;
+
+                    return new FieldRange(start + fieldOff, len);
+                }
+                else
+                    return new FieldRange(in.readInt(), in.readInt());
+            }
+            else {
+                fieldOff += len;
+
+                if (isHandle) {
+                    in.skipBytes(8);
+                    fieldOff += 8;
+                }
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     *
+     */
+    private static class FieldRange {
+        /** */
+        private int start;
+
+        /** */
+        private int len;
+
+        /**
+         * @param start Start.
+         * @param len   Length.
+         */
+        public FieldRange(int start, int len) {
+            this.start = start;
+            this.len = len;
+        }
+    }
     /** {@inheritDoc} */
     @Override public void registerValidation(ObjectInputValidation obj, int pri) {
         // No-op.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
new file mode 100644
index 0000000..34ca279
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectInputStreamExt.java
@@ -0,0 +1,51 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.internal.util.io.*;
+
+import java.io.*;
+
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
+import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerExt.*;
+
+
+/**
+ * TODO: IGNITE-950
+ */
+public class OptimizedObjectInputStreamExt extends OptimizedObjectInputStream {
+    /** {@inheritDoc} */
+    public OptimizedObjectInputStreamExt(GridDataInput in) throws IOException {
+        super(in);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void skipFooter(Class<?> cls) throws IOException {
+        if (!ctx.isSystemType(cls.getName()) && metaHandler != null &&
+            metaHandler.metadata(resolveTypeId(cls.getName(), mapper)) != null) {
+            short footerLen = in.readShort();
+
+            if (footerLen != EMPTY_FOOTER)
+                in.skipBytes(footerLen - 2);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int readFieldType() throws IOException {
+        return in.readByte();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
new file mode 100644
index 0000000..9a5463b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadata.java
@@ -0,0 +1,172 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Metadata that keeps fields information. Used in conjunction with the footer that is added to some objects during
+ * marshalling.
+ */
+public class OptimizedObjectMetadata implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private List<FieldInfo> fieldsInfo;
+
+    /** Constructor. */
+    public OptimizedObjectMetadata() {
+        // No-op
+    }
+
+    /**
+     * Adds meta for a new field.
+     *
+     * @param fieldId Field ID.
+     * @param fieldType Field type.
+     */
+    public void addMeta(int fieldId, OptimizedFieldType fieldType) {
+        if (fieldsInfo == null)
+            fieldsInfo = new ArrayList<>();
+
+
+
+        fieldsInfo.add(new FieldInfo(fieldId, fieldType));
+    }
+
+    /**
+     * Gets {@link OptimizedObjectMetadata.FieldInfo} at the {@code index}.
+     *
+     * @param index Position.
+     * @return Field meta info.
+     */
+    public FieldInfo getMeta(int index) {
+        return fieldsInfo.get(index);
+    }
+    /**
+     * Returns all the metadata stored for the object.
+     *
+     * @return Metadata collection.
+     */
+    public List<FieldInfo> getMeta() {
+        return Collections.unmodifiableList(fieldsInfo);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        if (fieldsInfo == null) {
+            out.writeInt(0);
+            return;
+        }
+
+        out.writeInt(fieldsInfo.size());
+
+        for (FieldInfo fieldInfo : fieldsInfo) {
+            out.writeInt(fieldInfo.id);
+            out.writeByte(fieldInfo.type.ordinal());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        int size = in.readInt();
+
+        fieldsInfo = new ArrayList<>(size);
+
+        for (int i = 0; i < size; i++)
+            fieldsInfo.add(new FieldInfo(in.readInt(), OptimizedFieldType.values()[in.readByte()]));
+    }
+
+    /**
+     * Field info.
+     */
+    public static class FieldInfo {
+        /** Field ID. */
+        int id;
+
+        /** Field len. */
+        int len;
+
+        /** Field type. */
+        OptimizedFieldType type;
+
+        /**
+         * Constructor.
+         *
+         * @param id Field ID.
+         * @param type Field len.
+         */
+        public FieldInfo(int id, OptimizedFieldType type) {
+            this.id = id;
+            this.type = type;
+
+            len = 1;
+
+            switch (type) {
+                case BYTE:
+                case BOOLEAN:
+                    len += 1;
+                    break;
+
+                case SHORT:
+                case CHAR:
+                    len += 2;
+                    break;
+
+                case INT:
+                case FLOAT:
+                    len += 4;
+                    break;
+
+                case LONG:
+                case DOUBLE:
+                    len += 8;
+                    break;
+
+                case OTHER:
+                    len = OptimizedMarshallerExt.VARIABLE_LEN;
+                    break;
+
+                default:
+                    throw new IgniteException("Unknown field type: " + type);
+            }
+
+            assert len != 1;
+        }
+
+        /**
+         * Returns ID.
+         * @return ID.
+         */
+        public int id() {
+            return id;
+        }
+
+        /**
+         * Returns length.
+         * @return Lenght.
+         */
+        public int length() {
+            return len;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java
new file mode 100644
index 0000000..ee85754
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectMetadataKey.java
@@ -0,0 +1,70 @@
+/*
+ * 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.marshaller.optimized;
+
+import org.apache.ignite.internal.processors.cache.*;
+
+import java.io.*;
+
+/**
+ * Optimized object metadata key.
+ */
+public class OptimizedObjectMetadataKey extends GridCacheUtilityKey<OptimizedObjectMetadataKey>
+    implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private int typeId;
+
+    /**
+     * For {@link Externalizable}.
+     */
+    public OptimizedObjectMetadataKey() {
+        // No-op
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param typeId Type id.
+     */
+    public OptimizedObjectMetadataKey(int typeId) {
+        this.typeId = typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeInt(typeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        typeId = in.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean equalsx(OptimizedObjectMetadataKey key) {
+        return typeId == key.typeId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return typeId;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5749b068/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
index 84b8351..1702b11 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedObjectOutputStream.java
@@ -23,10 +23,12 @@ import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.lang.reflect.*;
 import java.util.*;
+import java.util.Date;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
@@ -34,7 +36,7 @@ import static org.apache.ignite.marshaller.optimized.OptimizedMarshallerUtils.*;
 /**
  * Optimized object output stream.
  */
-public class OptimizedObjectOutputStream extends ObjectOutputStream {
+public class OptimizedObjectOutputStream extends ObjectOutputStream implements OptimizedFieldsWriter {
     /** */
     private static final Collection<String> CONVERTED_ERR = F.asList(
         "weblogic/management/ManagementException",
@@ -55,6 +57,9 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     protected ConcurrentMap<Class, OptimizedClassDescriptor> clsMap;
 
     /** */
+    protected OptimizedMarshallerMetaHandler metaHandler;
+
+    /** */
     protected boolean requireSer;
 
     /** */
@@ -72,6 +77,9 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     /** */
     private PutFieldImpl curPut;
 
+    /** */
+    private Stack<Footer> marshalAwareFooters;
+
     /**
      * @param out Output.
      * @throws IOException In case of error.
@@ -97,6 +105,23 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     }
 
     /**
+     * @param clsMap Class descriptors by class map.
+     * @param ctx Context.
+     * @param mapper ID mapper.
+     * @param requireSer Require {@link Serializable} flag.
+     * @param metaHandler Metadata handler.
+     */
+    protected void context(ConcurrentMap<Class, OptimizedClassDescriptor> clsMap,
+        MarshallerContext ctx,
+        OptimizedMarshallerIdMapper mapper,
+        boolean requireSer,
+        OptimizedMarshallerMetaHandler metaHandler) {
+        context(clsMap, ctx, mapper, requireSer);
+
+        this.metaHandler = metaHandler;
+    }
+
+    /**
      * @return Require {@link Serializable} flag.
      */
     boolean requireSerializable() {
@@ -116,6 +141,7 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
 
         ctx = null;
         clsMap = null;
+        metaHandler = null;
     }
 
     /** {@inheritDoc} */
@@ -308,6 +334,36 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
     }
 
     /**
+     * Writes marshal aware object with footer injected to the end of the stream.
+     *
+     * @param obj Object.
+     * @throws IOException In case of error.
+     */
+    void writeMarshalAware(Object obj) throws IOException {
+        Footer footer = createFooter(obj.getClass());
+
+        if (footer == null)
+            throw new IOException("Failed to marshal OptimizedMarshalAware object. OptimizedMarshallerExt must be " +
+                "set to IgniteConfiguration [obj=" + obj.getClass().getName() + "]");
+
+        if (marshalAwareFooters == null)
+            marshalAwareFooters = new Stack<>();
+
+        marshalAwareFooters.push(footer);
+
+        OptimizedMarshalAware marshalAwareObj = (OptimizedMarshalAware)obj;
+
+        marshalAwareObj.writeFields(this);
+
+        footer.write();
+
+        marshalAwareFooters.pop();
+
+        if (marshalAwareFooters.empty())
+            marshalAwareFooters = null;
+    }
+
+    /**
      * Writes serializable object.
      *
      * @param obj Object.
@@ -840,6 +896,7 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
         curObj = null;
         curFields = null;
         curPut = null;
+        marshalAwareFooters = null;
     }
 
     /** {@inheritDoc} */
@@ -862,6 +919,156 @@ public class OptimizedObjectOutputStream extends ObjectOutputStream {
         // No-op
     }
 
+    /** {@inheritDoc} */
+    @Override public void writeByte(String fieldName, byte val) throws IOException {
+        writeFieldType(BYTE);
+        out.writeByte(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.BYTE, 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(String fieldName, short val) throws IOException {
+        writeFieldType(SHORT);
+        out.writeShort(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.SHORT, 2);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(String fieldName, int val) throws IOException {
+        writeFieldType(INT);
+        out.writeInt(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.INT, 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(String fieldName, long val) throws IOException {
+        writeFieldType(LONG);
+        out.writeLong(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.LONG, 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(String fieldName, float val) throws IOException {
+        writeFieldType(FLOAT);
+        out.writeFloat(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.FLOAT, 4);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(String fieldName, double val) throws IOException {
+        writeFieldType(DOUBLE);
+        out.writeDouble(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.DOUBLE, 8);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(String fieldName, char val) throws IOException {
+        writeFieldType(CHAR);
+        out.writeChar(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.CHAR, 2);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(String fieldName, boolean val) throws IOException {
+        writeFieldType(BOOLEAN);
+        out.writeBoolean(val);
+        putFieldToFooter(fieldName, OptimizedFieldType.BOOLEAN, 1);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeString(String fieldName, @Nullable String val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObject(String fieldName, @Nullable Object obj) throws IOException {
+        int pos = out.offset();
+
+        writeObject(obj);
+        putFieldToFooter(fieldName, OptimizedFieldType.OTHER, out.offset() - pos);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByteArray(String fieldName, @Nullable byte[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShortArray(String fieldName, @Nullable short[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeIntArray(String fieldName, @Nullable int[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLongArray(String fieldName, @Nullable long[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloatArray(String fieldName, @Nullable float[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDoubleArray(String fieldName, @Nullable double[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeCharArray(String fieldName, @Nullable char[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBooleanArray(String fieldName, @Nullable boolean[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeStringArray(String fieldName, @Nullable String[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeObjectArray(String fieldName, @Nullable Object[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> void writeCollection(String fieldName, @Nullable Collection<T> col) throws IOException {
+        writeObject(fieldName, col);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> void writeMap(String fieldName, @Nullable Map<K, V> map) throws IOException {
+        writeObject(fieldName, map);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnum(String fieldName, T val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T extends Enum<?>> void writeEnumArray(String fieldName, T[] val) throws IOException {
+        writeObject(fieldName, val);
+    }
+
+    /**
+     * Puts field to the footer.
+     *
+     * @param fieldName Field name.
+     * @param type Field type.
+     * @param len Field length.
+     */
+    private void putFieldToFooter(String fieldName, OptimizedFieldType type, int len) {
+        marshalAwareFooters.peek().put(OptimizedMarshallerUtils.resolveFieldId(fieldName), type, len);
+    }
+
     /**
      * Creates new instance of {@code Footer}.
      *


[20/33] incubator-ignite git commit: Merge branches 'ignite-973' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6

Posted by sb...@apache.org.
Merge branches 'ignite-973' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6


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

Branch: refs/heads/ignite-950
Commit: 1cc00000a641f3d52964710377a8c29d9ff59947
Parents: 309eeb0 774bc94
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:47:29 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:47:29 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 ++-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 179 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 7 files changed, 233 insertions(+), 40 deletions(-)
----------------------------------------------------------------------



[19/33] incubator-ignite git commit: #ignite-973 - minor

Posted by sb...@apache.org.
#ignite-973 - minor


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

Branch: refs/heads/ignite-950
Commit: 774bc94ff7491735cf966f6f7a9b2a1887071dcf
Parents: ced62b8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:46:08 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:46:08 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheOffheapEvictQueryTest.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/774bc94f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index fc6c125..72bfcaa 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -69,7 +69,7 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
      */
     public void testEvictAndRemove() throws Exception {
         final int KEYS_CNT = 3000;
-        final int THREADS_CNT = 256;
+        final int THREADS_CNT = 50;
 
         final IgniteCache<Integer,Integer> c = startGrid().cache(null);
 


[17/33] incubator-ignite git commit: # ignite-sprint-6 do not call ping from exchange worker (cherry picked from commit 8fa9d3d)

Posted by sb...@apache.org.
# ignite-sprint-6 do not call ping from exchange worker (cherry picked from commit 8fa9d3d)


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

Branch: refs/heads/ignite-950
Commit: 309eeb0127546c2775f8514e1759497fccf1c8b4
Parents: 415264e
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 15:37:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 09:24:54 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 64 ++++++++++++------
 .../GridCachePartitionExchangeManager.java      | 70 +++++++++-----------
 .../GridCacheAbstractFailoverSelfTest.java      |  6 +-
 3 files changed, 79 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/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 eef9fde..74a4512 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
@@ -323,7 +323,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * Processes failed messages.
      * @param nodeId niode id.
      * @param msg message.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
@@ -511,6 +511,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
      */
+    @SuppressWarnings("unchecked")
     private IgniteInternalFuture<Object> startFuture(GridCacheMessage cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
@@ -574,6 +575,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      * @throws ClusterTopologyCheckedException If receiver left.
      */
@@ -734,6 +736,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param nodeId ID of node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      */
     public void send(UUID nodeId, GridCacheMessage msg, GridIoPolicy plc) throws IgniteCheckedException {
@@ -795,8 +798,41 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Sends message without retries and node ping in case of error.
+     *
+     * @param node Node to send message to.
+     * @param msg Message.
+     * @param plc IO policy.
+     * @throws IgniteCheckedException If send failed.
+     */
+    public void sendNoRetry(ClusterNode node,
+        GridCacheMessage msg,
+        GridIoPolicy plc)
+        throws IgniteCheckedException
+    {
+        assert node != null;
+        assert msg != null;
+
+        onSend(msg, null);
+
+        try {
+            cctx.gridIO().send(node, TOPIC_CACHE, msg, plc);
+
+            if (log.isDebugEnabled())
+                log.debug("Sent cache message [msg=" + msg + ", node=" + U.toShortString(node) + ']');
+        }
+        catch (IgniteCheckedException e) {
+            if (!cctx.discovery().alive(node.id()))
+                throw new ClusterTopologyCheckedException("Node left grid while sending message to: " + node.id(), e);
+            else
+                throw e;
+        }
+    }
+
+    /**
      * Adds message handler.
      *
+     * @param cacheId Cache ID.
      * @param type Type of message.
      * @param c Handler.
      */
@@ -846,29 +882,15 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         idxClsHandlers.remove(cacheId);
 
-        for (Iterator<ListenerKey> iterator = clsHandlers.keySet().iterator(); iterator.hasNext(); ) {
-            ListenerKey key = iterator.next();
+        for (Iterator<ListenerKey> iter = clsHandlers.keySet().iterator(); iter.hasNext(); ) {
+            ListenerKey key = iter.next();
 
             if (key.cacheId == cacheId)
-                iterator.remove();
+                iter.remove();
         }
     }
 
     /**
-     * @param lsnr Listener to add.
-     */
-    public void addDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().addDisconnectListener(lsnr);
-    }
-
-    /**
-     * @param lsnr Listener to remove.
-     */
-    public void removeDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().removeDisconnectListener(lsnr);
-    }
-
-    /**
      * @param msgCls Message class to check.
      * @return Message index.
      */
@@ -1029,11 +1051,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = cacheId;
+            int res = cacheId;
 
-            result = 31 * result + msgCls.hashCode();
+            res = 31 * res + msgCls.hashCode();
 
-            return result;
+            return res;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/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 ad4cf50..edd0ad7 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
@@ -567,27 +567,21 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         Collection<ClusterNode> rmts = null;
 
-        try {
-            // If this is the oldest node.
-            if (oldest.id().equals(cctx.localNodeId())) {
-                rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
-
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
+        // If this is the oldest node.
+        if (oldest.id().equals(cctx.localNodeId())) {
+            rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
 
-                sendAllPartitions(rmts);
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing local partitions from non-oldest node: " +
-                        cctx.localNodeId());
+            if (log.isDebugEnabled())
+                log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
 
-                sendLocalPartitions(oldest, null);
-            }
+            sendAllPartitions(rmts);
         }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to refresh partition map [oldest=" + oldest.id() + ", rmts=" + U.nodeIds(rmts) +
-                ", loc=" + cctx.localNodeId() + ']', e);
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Refreshing local partitions from non-oldest node: " +
+                    cctx.localNodeId());
+
+            sendLocalPartitions(oldest, null);
         }
     }
 
@@ -616,10 +610,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param nodes Nodes.
      * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes)
-        throws IgniteCheckedException {
+    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes) {
         GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE);
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
@@ -634,7 +626,19 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
-        cctx.io().safeSend(nodes, m, SYSTEM_POOL, null);
+        for (ClusterNode node : nodes) {
+            try {
+                cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
+            }
+            catch (ClusterTopologyCheckedException ignore) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
+                        node.id() + ", msg=" + m + ']');
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send partitions full message [node=" + node + ']', e);
+            }
+        }
 
         return true;
     }
@@ -642,11 +646,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param node Node.
      * @param id ID.
-     * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
-        throws IgniteCheckedException {
+    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
             cctx.kernalContext().clientNode(),
             cctx.versions().last());
@@ -669,16 +670,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
 
         try {
-            cctx.io().send(node, m, SYSTEM_POOL);
-
-            return true;
+            cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
                     node.id() + ", msg=" + m + ']');
-
-            return false;
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send local partition map to node [node=" + node + ", exchId=" + id + ']', e);
         }
     }
 
@@ -903,13 +903,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
 
         try {
-            try {
-                sendLocalPartitions(node, msg.exchangeId());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send local partition map to node [nodeId=" + node.id() + ", exchId=" +
-                    msg.exchangeId() + ']', e);
-            }
+            sendLocalPartitions(node, msg.exchangeId());
         }
         finally {
             leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index b6cd88e..3c74674 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -74,9 +74,11 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
-        discoSpi.setSocketTimeout(10_000);
-        discoSpi.setAckTimeout(10_000);
+        discoSpi.setSocketTimeout(30_000);
+        discoSpi.setAckTimeout(30_000);
         discoSpi.setNetworkTimeout(60_000);
+        discoSpi.setHeartbeatFrequency(30_000);
+        discoSpi.setReconnectCount(2);
 
         return cfg;
     }


[06/33] incubator-ignite git commit: # ignite-1034

Posted by sb...@apache.org.
# ignite-1034


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

Branch: refs/heads/ignite-950
Commit: d34f3294fbe120c443448e4df201fc9337725706
Parents: 895771a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 13:27:26 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 13:27:26 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/communication/tcp/TcpCommunicationSpi.java       | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d34f3294/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index bbb9b1c..a540b7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -611,11 +611,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @LoggerResource
     private IgniteLogger log;
 
-    /** Ignite. */
-    @IgniteInstanceResource
-    @GridToStringExclude
-    private Ignite ignite;
-
     /** Local IP address. */
     private String locAddr;