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 2017/01/17 14:04:37 UTC

[01/50] [abbrv] ignite git commit: IGNITE-4459: Hadoop: weighted planned is default one from now on. This closes #1391.

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-11810-1 fcfd62ace -> 8f005c308


IGNITE-4459: Hadoop: weighted planned is default one from now on. This closes #1391.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: a9b1fc2b3840d47d7c978d9296e8ae6bdeb10be5
Parents: 2e691d8
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Dec 29 11:07:22 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 29 11:07:22 2016 +0300

----------------------------------------------------------------------
 .../mapreduce/IgniteHadoopMapReducePlanner.java | 414 -------------
 .../processors/hadoop/HadoopProcessor.java      |   4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 619 -------------------
 .../testsuites/IgniteHadoopTestSuite.java       |   2 -
 4 files changed, 2 insertions(+), 1037 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b1fc2b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
deleted file mode 100644
index ac42381..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopMapReducePlanner.java
+++ /dev/null
@@ -1,414 +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.hadoop.mapreduce;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.igfs.IgfsBlockLocation;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.IgniteEx;
-import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
-import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan;
-import org.apache.ignite.internal.processors.igfs.IgfsEx;
-import org.apache.ignite.internal.util.typedef.F;
-import org.jetbrains.annotations.NotNull;
-import org.jetbrains.annotations.Nullable;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.apache.ignite.IgniteFileSystem.IGFS_SCHEME;
-
-/**
- * Default map-reduce planner implementation.
- */
-public class IgniteHadoopMapReducePlanner extends HadoopAbstractMapReducePlanner {
-    /** {@inheritDoc} */
-    @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
-        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException {
-        // Convert collection of topology nodes to collection of topology node IDs.
-        Collection<UUID> topIds = new HashSet<>(top.size(), 1.0f);
-
-        for (ClusterNode topNode : top)
-            topIds.add(topNode.id());
-
-        Map<UUID, Collection<HadoopInputSplit>> mappers = mappers(top, topIds, job.input());
-
-        int rdcCnt = job.info().reducers();
-
-        if (rdcCnt < 0)
-            throw new IgniteCheckedException("Number of reducers must be non-negative, actual: " + rdcCnt);
-
-        Map<UUID, int[]> reducers = reducers(top, mappers, rdcCnt);
-
-        return new HadoopDefaultMapReducePlan(mappers, reducers);
-    }
-
-    /**
-     * Create plan for mappers.
-     *
-     * @param top Topology nodes.
-     * @param topIds Topology node IDs.
-     * @param splits Splits.
-     * @return Mappers map.
-     * @throws IgniteCheckedException If failed.
-     */
-    private Map<UUID, Collection<HadoopInputSplit>> mappers(Collection<ClusterNode> top, Collection<UUID> topIds,
-        Iterable<HadoopInputSplit> splits) throws IgniteCheckedException {
-        Map<UUID, Collection<HadoopInputSplit>> mappers = new HashMap<>();
-
-        Map<String, Collection<UUID>> nodes = groupByHost(top);
-
-        Map<UUID, Integer> nodeLoads = new HashMap<>(top.size(), 1.0f); // Track node load.
-
-        for (UUID nodeId : topIds)
-            nodeLoads.put(nodeId, 0);
-
-        for (HadoopInputSplit split : splits) {
-            UUID nodeId = nodeForSplit(split, topIds, nodes, nodeLoads);
-
-            if (log.isDebugEnabled())
-                log.debug("Mapped split to node [split=" + split + ", nodeId=" + nodeId + ']');
-
-            Collection<HadoopInputSplit> nodeSplits = mappers.get(nodeId);
-
-            if (nodeSplits == null) {
-                nodeSplits = new ArrayList<>();
-
-                mappers.put(nodeId, nodeSplits);
-            }
-
-            nodeSplits.add(split);
-
-            // Updated node load.
-            nodeLoads.put(nodeId, nodeLoads.get(nodeId) + 1);
-        }
-
-        return mappers;
-    }
-
-    /**
-     * Determine the best node for this split.
-     *
-     * @param split Split.
-     * @param topIds Topology node IDs.
-     * @param nodes Nodes.
-     * @param nodeLoads Node load tracker.
-     * @return Node ID.
-     * @throws IgniteCheckedException On error.
-     */
-    @SuppressWarnings("unchecked")
-    private UUID nodeForSplit(HadoopInputSplit split, Collection<UUID> topIds, Map<String, Collection<UUID>> nodes,
-        Map<UUID, Integer> nodeLoads) throws IgniteCheckedException {
-        if (split instanceof HadoopFileBlock) {
-            HadoopFileBlock split0 = (HadoopFileBlock)split;
-
-            if (IGFS_SCHEME.equalsIgnoreCase(split0.file().getScheme())) {
-                HadoopIgfsEndpoint endpoint = new HadoopIgfsEndpoint(split0.file().getAuthority());
-
-                IgfsEx igfs = (IgfsEx)((IgniteEx)ignite).igfsx(endpoint.igfs());
-
-                if (igfs != null && !igfs.isProxy(split0.file())) {
-                    IgfsPath path = new IgfsPath(split0.file());
-
-                    if (igfs.exists(path)) {
-                        Collection<IgfsBlockLocation> blocks;
-
-                        try {
-                            blocks = igfs.affinity(path, split0.start(), split0.length());
-                        }
-                        catch (IgniteException e) {
-                            throw new IgniteCheckedException(e);
-                        }
-
-                        assert blocks != null;
-
-                        if (blocks.size() == 1)
-                            // Fast-path, split consists of one IGFS block (as in most cases).
-                            return bestNode(blocks.iterator().next().nodeIds(), topIds, nodeLoads, false);
-                        else {
-                            // Slow-path, file consists of multiple IGFS blocks. First, find the most co-located nodes.
-                            Map<UUID, Long> nodeMap = new HashMap<>();
-
-                            List<UUID> bestNodeIds = null;
-                            long bestLen = -1L;
-
-                            for (IgfsBlockLocation block : blocks) {
-                                for (UUID blockNodeId : block.nodeIds()) {
-                                    if (topIds.contains(blockNodeId)) {
-                                        Long oldLen = nodeMap.get(blockNodeId);
-                                        long newLen = oldLen == null ? block.length() : oldLen + block.length();
-
-                                        nodeMap.put(blockNodeId, newLen);
-
-                                        if (bestNodeIds == null || bestLen < newLen) {
-                                            bestNodeIds = new ArrayList<>(1);
-
-                                            bestNodeIds.add(blockNodeId);
-
-                                            bestLen = newLen;
-                                        }
-                                        else if (bestLen == newLen) {
-                                            assert !F.isEmpty(bestNodeIds);
-
-                                            bestNodeIds.add(blockNodeId);
-                                        }
-                                    }
-                                }
-                            }
-
-                            if (bestNodeIds != null) {
-                                return bestNodeIds.size() == 1 ? bestNodeIds.get(0) :
-                                    bestNode(bestNodeIds, topIds, nodeLoads, true);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-
-        // Cannot use local IGFS for some reason, try selecting the node by host.
-        Collection<UUID> blockNodes = null;
-
-        for (String host : split.hosts()) {
-            Collection<UUID> hostNodes = nodes.get(host);
-
-            if (!F.isEmpty(hostNodes)) {
-                if (blockNodes == null)
-                    blockNodes = new ArrayList<>(hostNodes);
-                else
-                    blockNodes.addAll(hostNodes);
-            }
-        }
-
-        return bestNode(blockNodes, topIds, nodeLoads, false);
-    }
-
-    /**
-     * Finds the best (the least loaded) node among the candidates.
-     *
-     * @param candidates Candidates.
-     * @param topIds Topology node IDs.
-     * @param nodeLoads Known node loads.
-     * @param skipTopCheck Whether to skip topology check.
-     * @return The best node.
-     */
-    private UUID bestNode(@Nullable Collection<UUID> candidates, Collection<UUID> topIds, Map<UUID, Integer> nodeLoads,
-        boolean skipTopCheck) {
-        UUID bestNode = null;
-        int bestLoad = Integer.MAX_VALUE;
-
-        if (candidates != null) {
-            for (UUID candidate : candidates) {
-                if (skipTopCheck || topIds.contains(candidate)) {
-                    int load = nodeLoads.get(candidate);
-
-                    if (bestNode == null || bestLoad > load) {
-                        bestNode = candidate;
-                        bestLoad = load;
-
-                        if (bestLoad == 0)
-                            break; // Minimum load possible, no need for further iterations.
-                    }
-                }
-            }
-        }
-
-        if (bestNode == null) {
-            // Blocks are located on nodes which are not Hadoop-enabled, assign to the least loaded one.
-            bestLoad = Integer.MAX_VALUE;
-
-            for (UUID nodeId : topIds) {
-                int load = nodeLoads.get(nodeId);
-
-                if (bestNode == null || bestLoad > load) {
-                    bestNode = nodeId;
-                    bestLoad = load;
-
-                    if (bestLoad == 0)
-                        break; // Minimum load possible, no need for further iterations.
-                }
-            }
-        }
-
-        assert bestNode != null;
-
-        return bestNode;
-    }
-
-    /**
-     * Create plan for reducers.
-     *
-     * @param top Topology.
-     * @param mappers Mappers map.
-     * @param reducerCnt Reducers count.
-     * @return Reducers map.
-     */
-    private Map<UUID, int[]> reducers(Collection<ClusterNode> top,
-        Map<UUID, Collection<HadoopInputSplit>> mappers, int reducerCnt) {
-        // Determine initial node weights.
-        int totalWeight = 0;
-
-        List<WeightedNode> nodes = new ArrayList<>(top.size());
-
-        for (ClusterNode node : top) {
-            Collection<HadoopInputSplit> split = mappers.get(node.id());
-
-            int weight = reducerNodeWeight(node, split != null ? split.size() : 0);
-
-            nodes.add(new WeightedNode(node.id(), weight, weight));
-
-            totalWeight += weight;
-        }
-
-        // Adjust weights.
-        int totalAdjustedWeight = 0;
-
-        for (WeightedNode node : nodes) {
-            node.floatWeight = ((float)node.weight * reducerCnt) / totalWeight;
-
-            node.weight = Math.round(node.floatWeight);
-
-            totalAdjustedWeight += node.weight;
-        }
-
-        // Apply redundant/lost reducers.
-        Collections.sort(nodes);
-
-        if (totalAdjustedWeight > reducerCnt) {
-            // Too much reducers set.
-            ListIterator<WeightedNode> iter = nodes.listIterator(nodes.size() - 1);
-
-            while (totalAdjustedWeight != reducerCnt) {
-                if (!iter.hasPrevious())
-                    iter = nodes.listIterator(nodes.size() - 1);
-
-                WeightedNode node = iter.previous();
-
-                if (node.weight > 0) {
-                    node.weight -= 1;
-
-                    totalAdjustedWeight--;
-                }
-            }
-        }
-        else if (totalAdjustedWeight < reducerCnt) {
-            // Not enough reducers set.
-            ListIterator<WeightedNode> iter = nodes.listIterator(0);
-
-            while (totalAdjustedWeight != reducerCnt) {
-                if (!iter.hasNext())
-                    iter = nodes.listIterator(0);
-
-                WeightedNode node = iter.next();
-
-                if (node.floatWeight > 0.0f) {
-                    node.weight += 1;
-
-                    totalAdjustedWeight++;
-                }
-            }
-        }
-
-        int idx = 0;
-
-        Map<UUID, int[]> reducers = new HashMap<>(nodes.size(), 1.0f);
-
-        for (WeightedNode node : nodes) {
-            if (node.weight > 0) {
-                int[] arr = new int[node.weight];
-
-                for (int i = 0; i < arr.length; i++)
-                    arr[i] = idx++;
-
-                reducers.put(node.nodeId, arr);
-            }
-        }
-
-        return reducers;
-    }
-
-    /**
-     * Calculate node weight based on node metrics and data co-location.
-     *
-     * @param node Node.
-     * @param splitCnt Splits mapped to this node.
-     * @return Node weight.
-     */
-    @SuppressWarnings("UnusedParameters")
-    protected int reducerNodeWeight(ClusterNode node, int splitCnt) {
-        return splitCnt;
-    }
-
-    /**
-     * Weighted node.
-     */
-    private static class WeightedNode implements Comparable<WeightedNode> {
-        /** Node ID. */
-        private final UUID nodeId;
-
-        /** Weight. */
-        private int weight;
-
-        /** Floating point weight. */
-        private float floatWeight;
-
-        /**
-         * Constructor.
-         *
-         * @param nodeId Node ID.
-         * @param weight Weight.
-         * @param floatWeight Floating point weight.
-         */
-        private WeightedNode(UUID nodeId, int weight, float floatWeight) {
-            this.nodeId = nodeId;
-            this.weight = weight;
-            this.floatWeight = floatWeight;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            return obj != null && obj instanceof WeightedNode && F.eq(nodeId, ((WeightedNode)obj).nodeId);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return nodeId.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public int compareTo(@NotNull WeightedNode other) {
-            float res = other.floatWeight - floatWeight;
-
-            return res > 0.0f ? 1 : res < 0.0f ? -1 : nodeId.compareTo(other.nodeId);
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b1fc2b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
index f0df1e9..329d67f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.HadoopConfiguration;
-import org.apache.ignite.hadoop.mapreduce.IgniteHadoopMapReducePlanner;
+import org.apache.ignite.hadoop.mapreduce.IgniteHadoopWeightedMapReducePlanner;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
@@ -243,7 +243,7 @@ public class HadoopProcessor extends HadoopProcessorAdapter {
      */
     private void initializeDefaults(HadoopConfiguration cfg) {
         if (cfg.getMapReducePlanner() == null)
-            cfg.setMapReducePlanner(new IgniteHadoopMapReducePlanner());
+            cfg.setMapReducePlanner(new IgniteHadoopWeightedMapReducePlanner());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b1fc2b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java
deleted file mode 100644
index ee1c88f..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopDefaultMapReducePlannerSelfTest.java
+++ /dev/null
@@ -1,619 +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.hadoop.impl;
-
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteFileSystem;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.hadoop.mapreduce.IgniteHadoopMapReducePlanner;
-import org.apache.ignite.igfs.IgfsBlockLocation;
-import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner;
-import org.apache.ignite.internal.processors.igfs.IgfsBlockLocationImpl;
-import org.apache.ignite.internal.processors.igfs.IgfsIgniteMock;
-import org.apache.ignite.internal.processors.igfs.IgfsMock;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.testframework.GridTestNode;
-import org.apache.ignite.testframework.GridTestUtils;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-/**
- *
- */
-public class HadoopDefaultMapReducePlannerSelfTest extends HadoopAbstractSelfTest {
-    /** */
-    private static final UUID ID_1 = new UUID(0, 1);
-
-    /** */
-    private static final UUID ID_2 = new UUID(0, 2);
-
-    /** */
-    private static final UUID ID_3 = new UUID(0, 3);
-
-    /** */
-    private static final String HOST_1 = "host1";
-
-    /** */
-    private static final String HOST_2 = "host2";
-
-    /** */
-    private static final String HOST_3 = "host3";
-
-    /** */
-    private static final String INVALID_HOST_1 = "invalid_host1";
-
-    /** */
-    private static final String INVALID_HOST_2 = "invalid_host2";
-
-    /** */
-    private static final String INVALID_HOST_3 = "invalid_host3";
-
-    /** Mocked IGFS. */
-    private static final IgniteFileSystem IGFS = new MockIgfs();
-
-    /** Mocked Grid. */
-    private static final IgfsIgniteMock GRID = new IgfsIgniteMock(null, IGFS);
-
-    /** Planner. */
-    private static final HadoopMapReducePlanner PLANNER = new IgniteHadoopMapReducePlanner();
-
-    /** Block locations. */
-    private static final Map<Block, Collection<IgfsBlockLocation>> BLOCK_MAP = new HashMap<>();
-
-    /** Proxy map. */
-    private static final Map<URI, Boolean> PROXY_MAP = new HashMap<>();
-
-    /** Last created plan. */
-    private static final ThreadLocal<HadoopMapReducePlan> PLAN = new ThreadLocal<>();
-
-    /**
-     * Static initializer.
-     */
-    static {
-        GridTestUtils.setFieldValue(PLANNER, HadoopAbstractMapReducePlanner.class, "ignite", GRID);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        GridTestUtils.setFieldValue(PLANNER, HadoopAbstractMapReducePlanner.class, "log", log());
-
-        BLOCK_MAP.clear();
-        PROXY_MAP.clear();
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsOneBlockPerNode() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1);
-        HadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_2);
-        HadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_2));
-        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 2);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsOneBlockPerNode() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1);
-        HadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_2);
-        HadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 2);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) || ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureMappers(ID_3, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2);
-        HadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2);
-        HadoopFileBlock split3 = split(true, "/file3", 0, 100, HOST_1, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 100, ID_1, ID_2));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 100, ID_1, ID_2));
-        mapIgfsBlock(split3.file(), 0, 100, location(0, 100, ID_1, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsSeveralBlocksPerNode() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(false, "/file1", 0, 100, HOST_1, HOST_2);
-        HadoopFileBlock split2 = split(false, "/file2", 0, 100, HOST_1, HOST_2);
-        HadoopFileBlock split3 = split(false, "/file3", 0, 100, HOST_1, HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) || ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-
-        plan(3, split1, split2, split3);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testIgfsSeveralComplexBlocksPerNode() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(true, "/file1", 0, 100, HOST_1, HOST_2, HOST_3);
-        HadoopFileBlock split2 = split(true, "/file2", 0, 100, HOST_1, HOST_2, HOST_3);
-
-        mapIgfsBlock(split1.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_1, ID_3));
-        mapIgfsBlock(split2.file(), 0, 100, location(0, 50, ID_1, ID_2), location(51, 100, ID_2, ID_3));
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureEmpty(ID_2);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split2);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_1);
-        assert ensureEmpty(ID_3);
-
-        plan(1, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) || ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0);
-        assert ensureEmpty(ID_3);
-
-        plan(2, split1, split2);
-        assert ensureMappers(ID_1, split1);
-        assert ensureMappers(ID_2, split2);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureEmpty(ID_3);
-    }
-
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testNonIgfsOrphans() throws IgniteCheckedException {
-        HadoopFileBlock split1 = split(false, "/file1", 0, 100, INVALID_HOST_1, INVALID_HOST_2);
-        HadoopFileBlock split2 = split(false, "/file2", 0, 100, INVALID_HOST_1, INVALID_HOST_3);
-        HadoopFileBlock split3 = split(false, "/file3", 0, 100, INVALID_HOST_2, INVALID_HOST_3);
-
-        plan(1, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 1) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 1) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 1);
-
-        plan(2, split1);
-        assert ensureMappers(ID_1, split1) && ensureReducers(ID_1, 2) && ensureEmpty(ID_2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureMappers(ID_2, split1) && ensureReducers(ID_2, 2) && ensureEmpty(ID_3) ||
-            ensureEmpty(ID_1) && ensureEmpty(ID_2) && ensureMappers(ID_3, split1) && ensureReducers(ID_3, 2);
-
-        plan(1, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 0) ||
-            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 0) ||
-            ensureReducers(ID_1, 0) && ensureReducers(ID_2, 0) && ensureReducers(ID_3, 1);
-
-        plan(3, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1);
-        assert ensureReducers(ID_2, 1);
-        assert ensureReducers(ID_3, 1);
-
-        plan(5, split1, split2, split3);
-        assert ensureMappers(ID_1, split1) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split1) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split3) ||
-            ensureMappers(ID_1, split2) && ensureMappers(ID_2, split3) && ensureMappers(ID_3, split1) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split1) && ensureMappers(ID_3, split2) ||
-            ensureMappers(ID_1, split3) && ensureMappers(ID_2, split2) && ensureMappers(ID_3, split1);
-        assert ensureReducers(ID_1, 1) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 1) && ensureReducers(ID_3, 2) ||
-            ensureReducers(ID_1, 2) && ensureReducers(ID_2, 2) && ensureReducers(ID_3, 1);
-    }
-
-    /**
-     * Create plan.
-     *
-     * @param reducers Reducers count.
-     * @param splits Splits.
-     * @return Plan.
-     * @throws IgniteCheckedException If failed.
-     */
-    private static HadoopMapReducePlan plan(int reducers, HadoopInputSplit... splits) throws IgniteCheckedException {
-        assert reducers > 0;
-        assert splits != null && splits.length > 0;
-
-        Collection<HadoopInputSplit> splitList = new ArrayList<>(splits.length);
-
-        Collections.addAll(splitList, splits);
-
-        Collection<ClusterNode> top = new ArrayList<>();
-
-        GridTestNode node1 = new GridTestNode(ID_1);
-        GridTestNode node2 = new GridTestNode(ID_2);
-        GridTestNode node3 = new GridTestNode(ID_3);
-
-        node1.setHostName(HOST_1);
-        node2.setHostName(HOST_2);
-        node3.setHostName(HOST_3);
-
-        top.add(node1);
-        top.add(node2);
-        top.add(node3);
-
-        HadoopMapReducePlan plan = PLANNER.preparePlan(new HadoopPlannerMockJob(splitList, reducers), top, null);
-
-        PLAN.set(plan);
-
-        return plan;
-    }
-
-    /**
-     * Ensure that node contains the given mappers.
-     *
-     * @param nodeId Node ID.
-     * @param expSplits Expected splits.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureMappers(UUID nodeId, HadoopInputSplit... expSplits) {
-        Collection<HadoopInputSplit> expSplitsCol = new ArrayList<>();
-
-        Collections.addAll(expSplitsCol, expSplits);
-
-        Collection<HadoopInputSplit> splits = PLAN.get().mappers(nodeId);
-
-        return F.eq(expSplitsCol, splits);
-    }
-
-    /**
-     * Ensure that node contains the given amount of reducers.
-     *
-     * @param nodeId Node ID.
-     * @param reducers Reducers.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureReducers(UUID nodeId, int reducers) {
-        int[] reducersArr = PLAN.get().reducers(nodeId);
-
-        return reducers == 0 ? F.isEmpty(reducersArr) : (reducersArr != null && reducersArr.length == reducers);
-    }
-
-    /**
-     * Ensure that no mappers and reducers is located on this node.
-     *
-     * @param nodeId Node ID.
-     * @return {@code True} if this assumption is valid.
-     */
-    private static boolean ensureEmpty(UUID nodeId) {
-        return F.isEmpty(PLAN.get().mappers(nodeId)) && F.isEmpty(PLAN.get().reducers(nodeId));
-    }
-
-    /**
-     * Create split.
-     *
-     * @param igfs IGFS flag.
-     * @param file File.
-     * @param start Start.
-     * @param len Length.
-     * @param hosts Hosts.
-     * @return Split.
-     */
-    private static HadoopFileBlock split(boolean igfs, String file, long start, long len, String... hosts) {
-        URI uri = URI.create((igfs ? "igfs://igfs@" : "hdfs://") + file);
-
-        return new HadoopFileBlock(hosts, uri, start, len);
-    }
-
-    /**
-     * Create block location.
-     *
-     * @param start Start.
-     * @param len Length.
-     * @param nodeIds Node IDs.
-     * @return Block location.
-     */
-    private static IgfsBlockLocation location(long start, long len, UUID... nodeIds) {
-        assert nodeIds != null && nodeIds.length > 0;
-
-        Collection<ClusterNode> nodes = new ArrayList<>(nodeIds.length);
-
-        for (UUID id : nodeIds)
-            nodes.add(new GridTestNode(id));
-
-        return new IgfsBlockLocationImpl(start, len, nodes);
-    }
-
-    /**
-     * Map IGFS block to nodes.
-     *
-     * @param file File.
-     * @param start Start.
-     * @param len Length.
-     * @param locations Locations.
-     */
-    private static void mapIgfsBlock(URI file, long start, long len, IgfsBlockLocation... locations) {
-        assert locations != null && locations.length > 0;
-
-        IgfsPath path = new IgfsPath(file);
-
-        Block block = new Block(path, start, len);
-
-        Collection<IgfsBlockLocation> locationsList = new ArrayList<>();
-
-        Collections.addAll(locationsList, locations);
-
-        BLOCK_MAP.put(block, locationsList);
-    }
-
-    /**
-     * Block.
-     */
-    private static class Block {
-        /** */
-        private final IgfsPath path;
-
-        /** */
-        private final long start;
-
-        /** */
-        private final long len;
-
-        /**
-         * Constructor.
-         *
-         * @param path Path.
-         * @param start Start.
-         * @param len Length.
-         */
-        private Block(IgfsPath path, long start, long len) {
-            this.path = path;
-            this.start = start;
-            this.len = len;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("RedundantIfStatement")
-        @Override public boolean equals(Object o) {
-            if (this == o) return true;
-            if (!(o instanceof Block)) return false;
-
-            Block block = (Block) o;
-
-            if (len != block.len)
-                return false;
-
-            if (start != block.start)
-                return false;
-
-            if (!path.equals(block.path))
-                return false;
-
-            return true;
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = path.hashCode();
-
-            res = 31 * res + (int) (start ^ (start >>> 32));
-            res = 31 * res + (int) (len ^ (len >>> 32));
-
-            return res;
-        }
-    }
-
-    /**
-     * Mocked IGFS.
-     */
-    private static class MockIgfs extends IgfsMock {
-        /**
-         * Constructor.
-         */
-        public MockIgfs() {
-            super("igfs");
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isProxy(URI path) {
-            return PROXY_MAP.containsKey(path) && PROXY_MAP.get(path);
-        }
-
-        /** {@inheritDoc} */
-        @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len) {
-            return BLOCK_MAP.get(new Block(path, start, len));
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean exists(IgfsPath path) {
-            return true;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9b1fc2b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 01893fb..6c2d5c4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -55,7 +55,6 @@ import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSy
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemLoopbackExternalSecondarySelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.igfs.IgniteHadoopFileSystemSecondaryFileSystemInitializationSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopCommandLineTest;
-import org.apache.ignite.internal.processors.hadoop.impl.HadoopDefaultMapReducePlannerSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopFileSystemsTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopGroupingTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopJobTrackerSelfTest;
@@ -116,7 +115,6 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(HadoopUserLibsSelfTest.class.getName())));
 
-        suite.addTest(new TestSuite(ldr.loadClass(HadoopDefaultMapReducePlannerSelfTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(HadoopWeightedMapReducePlannerTest.class.getName())));
 
         suite.addTest(new TestSuite(ldr.loadClass(BasicUserNameMapperSelfTest.class.getName())));


[19/50] [abbrv] ignite git commit: .NET: Fix flaky tests - improve assertions

Posted by sb...@apache.org.
.NET: Fix flaky tests - improve assertions


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

Branch: refs/heads/ignite-gg-11810-1
Commit: c4d67d2ab737c398c9158dfeb67dc979f5c052af
Parents: 87fbb27
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 16:07:36 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 16:07:36 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs | 2 ++
 .../platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs | 6 +++++-
 2 files changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c4d67d2a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
index 250f974..1239794 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTest.cs
@@ -309,6 +309,8 @@ namespace Apache.Ignite.Core.Tests.Cache
 
                 Ignition.Start(cfg);
             }
+
+            Assert.AreEqual(GridCount(), GetIgnite(0).GetCluster().GetNodes().Count);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/c4d67d2a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
index 82c6742..8b57c81 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
@@ -56,6 +56,8 @@ namespace Apache.Ignite.Core.Tests
             _grid1 = Ignition.Start(Configuration("config\\compute\\compute-grid1.xml"));
             _grid2 = Ignition.Start(Configuration("config\\compute\\compute-grid2.xml"));
             _grid3 = Ignition.Start(Configuration("config\\compute\\compute-grid3.xml"));
+
+            Assert.AreEqual(3, _grid1.GetCluster().GetNodes().Count);
         }
 
         /// <summary>
@@ -556,7 +558,9 @@ namespace Apache.Ignite.Core.Tests
             Func<IEnumerable<string>, IEnumerable<string>> resultFunc, int expectedRepeat)
         {
             // check if expected message count has been received; Wait returns false if there were none.
-            Assert.IsTrue(ReceivedEvent.Wait(MessageTimeout));
+            Assert.IsTrue(ReceivedEvent.Wait(MessageTimeout),
+                string.Format("expectedMessages: {0}, expectedRepeat: {1}, remaining: {2}",
+                    expectedMessages, expectedRepeat, ReceivedEvent.CurrentCount));
 
             expectedMessages = expectedMessages.SelectMany(x => Enumerable.Repeat(x, expectedRepeat));
 


[45/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
index 5fc3d25..c34296a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 
 /**
@@ -72,56 +73,56 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setForward(buf, 0);
-        setRemoveId(buf, 0);
+        setCount(pageAddr, 0);
+        setForward(pageAddr, 0);
+        setRemoveId(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Forward page ID.
      */
-    public final long getForward(ByteBuffer buf) {
-        return buf.getLong(FORWARD_OFF);
+    public final long getForward(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FORWARD_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Forward page ID.
      */
-    public final void setForward(ByteBuffer buf, long pageId) {
-        buf.putLong(FORWARD_OFF, pageId);
+    public final void setForward(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, FORWARD_OFF, pageId);
 
-        assert getForward(buf) == pageId;
+        assert getForward(pageAddr) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Remove ID.
      */
-    public final long getRemoveId(ByteBuffer buf) {
-        return buf.getLong(REMOVE_ID_OFF);
+    public final long getRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REMOVE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Remove ID.
      */
-    public final void setRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(REMOVE_ID_OFF, rmvId);
+    public final void setRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, REMOVE_ID_OFF, rmvId);
 
-        assert getRemoveId(buf) == rmvId;
+        assert getRemoveId(pageAddr) == rmvId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Items count in the page.
      */
-    public final int getCount(ByteBuffer buf) {
-        int cnt = buf.getShort(CNT_OFF) & 0xFFFF;
+    public final int getCount(long pageAddr) {
+        int cnt = PageUtils.getShort(pageAddr, CNT_OFF) & 0xFFFF;
 
         assert cnt >= 0: cnt;
 
@@ -129,20 +130,20 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Count.
      */
-    public final void setCount(ByteBuffer buf, int cnt) {
+    public final void setCount(long pageAddr, int cnt) {
         assert cnt >= 0: cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
 
-        assert getCount(buf) == cnt;
+        assert getCount(pageAddr) == cnt;
     }
 
     /**
      * @return {@code true} If we can get the full row from this page using
-     * method {@link BPlusTree#getRow(BPlusIO, ByteBuffer, int)}.
+     * method {@link BPlusTree#getRow(BPlusIO, long, int)}.
      * Must always be {@code true} for leaf pages.
      */
     public final boolean canGetRow() {
@@ -157,27 +158,28 @@ public abstract class BPlusIO<L> extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max items count.
      */
-    public abstract int getMaxCount(ByteBuffer buf);
+    public abstract int getMaxCount(long pageAddr, int pageSize);
 
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Lookup or full row.
      * @param rowBytes Row bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public final void store(ByteBuffer buf, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
+    public final void store(long pageAddr, int idx, L row, byte[] rowBytes) throws IgniteCheckedException {
         int off = offset(idx);
 
         if (rowBytes == null)
-            storeByOffset(buf, off, row);
+            storeByOffset(pageAddr, off, row);
         else
-            putBytes(buf, off, rowBytes);
+            putBytes(pageAddr, off, rowBytes);
     }
 
     /**
@@ -189,6 +191,17 @@ public abstract class BPlusIO<L> extends PageIO {
     /**
      * Store the needed info about the row in the page. Leaf and inner pages can store different info.
      *
+     * @param pageAddr Page address.
+     * @param off Offset in bytes.
+     * @param row Lookup or full row.
+     * @throws IgniteCheckedException If failed.
+     */
+    public abstract void storeByOffset(long pageAddr, int off, L row) throws IgniteCheckedException;
+
+
+    /**
+     * Store the needed info about the row in the page. Leaf and inner pages can store different info.
+     *
      * @param buf Buffer.
      * @param off Offset in bytes.
      * @param row Lookup or full row.
@@ -199,137 +212,141 @@ public abstract class BPlusIO<L> extends PageIO {
     /**
      * Store row info from the given source.
      *
-     * @param dst Destination buffer
+     * @param dstPageAddr Destination page address.
      * @param dstIdx Destination index.
      * @param srcIo Source IO.
-     * @param src Source buffer.
+     * @param srcPageAddr Source page address.
      * @param srcIdx Source index.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void store(ByteBuffer dst, int dstIdx, BPlusIO<L> srcIo, ByteBuffer src, int srcIdx)
+    public abstract void store(long dstPageAddr, int dstIdx, BPlusIO<L> srcIo, long srcPageAddr, int srcIdx)
         throws IgniteCheckedException;
 
     /**
      * Get lookup row.
      *
      * @param tree Tree.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Lookup row.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract L getLookupRow(BPlusTree<L, ?> tree, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    public abstract L getLookupRow(BPlusTree<L, ?> tree, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
-     * Copy items from source buffer to destination buffer.
+     * Copy items from source page to destination page.
      * Both pages must be of the same type and the same version.
      *
-     * @param src Source buffer.
-     * @param dst Destination buffer.
+     * @param srcPageAddr Source page address.
+     * @param dstPageAddr Destination page address.
      * @param srcIdx Source begin index.
      * @param dstIdx Destination begin index.
      * @param cnt Items count.
      * @param cpLeft Copy leftmost link (makes sense only for inner pages).
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
+    public abstract void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt, boolean cpLeft)
         throws IgniteCheckedException;
 
     // Methods for B+Tree logic.
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param row Row to insert.
      * @param rowBytes Row bytes.
      * @param rightId Page ID which will be to the right child for the inserted item.
      * @throws IgniteCheckedException If failed.
      */
-    public void insert(ByteBuffer buf, int idx, L row, byte[] rowBytes, long rightId)
+    public void insert(long pageAddr, int idx, L row, byte[] rowBytes, long rightId)
         throws IgniteCheckedException {
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         // Move right all the greater elements to make a free slot for a new row link.
-        copyItems(buf, buf, idx, idx + 1, cnt - idx, false);
+        copyItems(pageAddr, pageAddr, idx, idx + 1, cnt - idx, false);
 
-        setCount(buf, cnt + 1);
+        setCount(pageAddr, cnt + 1);
 
-        store(buf, idx, row, rowBytes);
+        store(pageAddr, idx, row, rowBytes);
     }
 
     /**
-     * @param buf Splitting buffer.
+     * @param pageAddr Splitting page address.
      * @param fwdId Forward page ID.
-     * @param fwdBuf Forward buffer.
+     * @param fwdPageAddr Forward page address.
      * @param mid Bisection index.
      * @param cnt Initial elements count in the page being split.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void splitForwardPage(
-        ByteBuffer buf,
+        long pageAddr,
         long fwdId,
-        ByteBuffer fwdBuf,
+        long fwdPageAddr,
         int mid,
-        int cnt
+        int cnt,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(fwdBuf, fwdId);
+        initNewPage(fwdPageAddr, fwdId, pageSize);
 
         cnt -= mid;
 
-        copyItems(buf, fwdBuf, mid, 0, cnt, true);
+        copyItems(pageAddr, fwdPageAddr, mid, 0, cnt, true);
 
-        setCount(fwdBuf, cnt);
-        setForward(fwdBuf, getForward(buf));
+        setCount(fwdPageAddr, cnt);
+        setForward(fwdPageAddr, getForward(pageAddr));
 
         // Copy remove ID to make sure that if inner remove touched this page, then retry
         // will happen even for newly allocated forward page.
-        setRemoveId(fwdBuf, getRemoveId(buf));
+        setRemoveId(fwdPageAddr, getRemoveId(pageAddr));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param mid Bisection index.
      * @param fwdId New forward page ID.
      */
-    public void splitExistingPage(ByteBuffer buf, int mid, long fwdId) {
-        setCount(buf, mid);
-        setForward(buf, fwdId);
+    public void splitExistingPage(long pageAddr, int mid, long fwdId) {
+        setCount(pageAddr, mid);
+        setForward(pageAddr, fwdId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @throws IgniteCheckedException If failed.
      */
-    public void remove(ByteBuffer buf, int idx, int cnt) throws IgniteCheckedException {
+    public void remove(long pageAddr, int idx, int cnt) throws IgniteCheckedException {
         cnt--;
 
-        copyItems(buf, buf, idx + 1, idx, cnt - idx, false);
-        setCount(buf, cnt);
+        copyItems(pageAddr, pageAddr, idx + 1, idx, cnt - idx, false);
+        setCount(pageAddr, cnt);
     }
 
     /**
      * @param prntIo Parent IO.
-     * @param prnt Parent buffer.
+     * @param prntPageAddr Parent page address.
      * @param prntIdx Split key index in parent.
-     * @param left Left buffer.
-     * @param right Right buffer.
+     * @param leftPageAddr Left page address.
+     * @param rightPageAddr Right page address.
      * @param emptyBranch We are merging an empty branch.
+     * @param pageSize Page size.
      * @return {@code false} If we were not able to merge.
      * @throws IgniteCheckedException If failed.
      */
     public boolean merge(
         BPlusIO<L> prntIo,
-        ByteBuffer prnt,
+        long prntPageAddr,
         int prntIdx,
-        ByteBuffer left,
-        ByteBuffer right,
-        boolean emptyBranch
+        long leftPageAddr,
+        long rightPageAddr,
+        boolean emptyBranch,
+        int pageSize
     ) throws IgniteCheckedException {
-        int prntCnt = prntIo.getCount(prnt);
-        int leftCnt = getCount(left);
-        int rightCnt = getCount(right);
+        int prntCnt = prntIo.getCount(prntPageAddr);
+        int leftCnt = getCount(leftPageAddr);
+        int rightCnt = getCount(rightPageAddr);
 
         int newCnt = leftCnt + rightCnt;
 
@@ -337,13 +354,13 @@ public abstract class BPlusIO<L> extends PageIO {
         if (!isLeaf() && !emptyBranch)
             newCnt++;
 
-        if (newCnt > getMaxCount(left)) {
+        if (newCnt > getMaxCount(leftPageAddr, pageSize)) {
             assert !emptyBranch;
 
             return false;
         }
 
-        setCount(left, newCnt);
+        setCount(leftPageAddr, newCnt);
 
         // Move down split key in inner pages.
         if (!isLeaf() && !emptyBranch) {
@@ -351,37 +368,29 @@ public abstract class BPlusIO<L> extends PageIO {
 
             // We can be sure that we have enough free space to store split key here,
             // because we've done remove already and did not release child locks.
-            store(left, leftCnt, prntIo, prnt, prntIdx);
+            store(leftPageAddr, leftCnt, prntIo, prntPageAddr, prntIdx);
 
             leftCnt++;
         }
 
-        copyItems(right, left, 0, leftCnt, rightCnt, !emptyBranch);
-        setForward(left, getForward(right));
+        copyItems(rightPageAddr, leftPageAddr, 0, leftCnt, rightCnt, !emptyBranch);
+        setForward(leftPageAddr, getForward(rightPageAddr));
 
-        long rmvId = getRemoveId(right);
+        long rmvId = getRemoveId(rightPageAddr);
 
         // Need to have maximum remove ID.
-        if (rmvId > getRemoveId(left))
-            setRemoveId(left, rmvId);
+        if (rmvId > getRemoveId(leftPageAddr))
+            setRemoveId(leftPageAddr, rmvId);
 
         return true;
     }
 
     /**
-     * @param buf Buffer.
-     * @param pos Position in buffer.
+     * @param pageAddr Page address.
+     * @param pos Position in page.
      * @param bytes Bytes.
      */
-    private static void putBytes(ByteBuffer buf, int pos, byte[] bytes) {
-        int oldPos = buf.position();
-
-        try {
-            buf.position(pos);
-            buf.put(bytes);
-        }
-        finally {
-            buf.position(oldPos);
-        }
+    private static void putBytes(long pageAddr, int pos, byte[] bytes) {
+        PageUtils.putBytes(pageAddr, pos, bytes);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
index 90b0f37..c7018bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusInnerIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
 /**
@@ -45,71 +45,71 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
         // The structure of the page is the following:
         // |ITEMS_OFF|w|A|x|B|y|C|z|
         // where capital letters are data items, lowercase letters are 8 byte page references.
-        return (buf.capacity() - ITEMS_OFF - 8) / (itemSize + 8);
+        return (pageSize - ITEMS_OFF - 8) / (itemSize + 8);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getLeft(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_LEFT));
+    public final long getLeft(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, (offset(idx, SHIFT_LEFT)));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setLeft(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_LEFT), pageId);
+    public final void setLeft(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_LEFT), pageId);
 
-        assert pageId == getLeft(buf, idx);
+        assert pageId == getLeft(pageAddr, idx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Page ID.
      */
-    public final long getRight(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx, SHIFT_RIGHT));
+    public final long getRight(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx, SHIFT_RIGHT));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param pageId Page ID.
      */
-    public final void setRight(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx, SHIFT_RIGHT), pageId);
+    private void setRight(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx, SHIFT_RIGHT), pageId);
 
-        assert pageId == getRight(buf, idx);
+        assert pageId == getRight(pageAddr, idx);
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
         cnt *= itemSize + 8; // From items to bytes.
 
         if (dstIdx > srcIdx) {
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
 
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
         }
         else {
             if (cpLeft)
-                dst.putLong(offset(dstIdx, SHIFT_LEFT), src.getLong(offset(srcIdx, SHIFT_LEFT)));
+                PageUtils.putLong(dstPageAddr, offset(dstIdx, SHIFT_LEFT), PageUtils.getLong(srcPageAddr, (offset(srcIdx, SHIFT_LEFT))));
 
-            PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt);
+            PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt);
         }
     }
 
@@ -131,39 +131,42 @@ public abstract class BPlusInnerIO<L> extends BPlusIO<L> {
 
     /** {@inheritDoc} */
     @Override public void insert(
-        ByteBuffer buf,
+        long pageAddr,
         int idx,
         L row,
         byte[] rowBytes,
         long rightId
     ) throws IgniteCheckedException {
-        super.insert(buf, idx, row, rowBytes, rightId);
+        super.insert(pageAddr, idx, row, rowBytes, rightId);
 
         // Setup reference to the right page on split.
-        setRight(buf, idx, rightId);
+        setRight(pageAddr, idx, rightId);
     }
 
     /**
-     * @param newRootBuf New root buffer.
+     * @param newRootPageAddr New root page address.
      * @param newRootId New root ID.
      * @param leftChildId Left child ID.
      * @param row Moved up row.
+     * @param rowBytes Bytes.
      * @param rightChildId Right child ID.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void initNewRoot(
-        ByteBuffer newRootBuf,
+        long newRootPageAddr,
         long newRootId,
         long leftChildId,
         L row,
         byte[] rowBytes,
-        long rightChildId
+        long rightChildId,
+        int pageSize
     ) throws IgniteCheckedException {
-        initNewPage(newRootBuf, newRootId);
+        initNewPage(newRootPageAddr, newRootId, pageSize);
 
-        setCount(newRootBuf, 1);
-        setLeft(newRootBuf, 0, leftChildId);
-        store(newRootBuf, 0, row, rowBytes);
-        setRight(newRootBuf, 0, rightChildId);
+        setCount(newRootPageAddr, 1);
+        setLeft(newRootPageAddr, 0, leftChildId);
+        store(newRootPageAddr, 0, row, rowBytes);
+        setRight(newRootPageAddr, 0, rightChildId);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
index 824c943..f3dccee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusLeafIO.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 
@@ -35,16 +34,16 @@ public abstract class BPlusLeafIO<L> extends BPlusIO<L> {
     }
 
     /** {@inheritDoc} */
-    @Override public int getMaxCount(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / itemSize;
+    @Override public int getMaxCount(long pageAddr, int pageSize) {
+        return (pageSize - ITEMS_OFF) / itemSize;
     }
 
     /** {@inheritDoc} */
-    @Override public final void copyItems(ByteBuffer src, ByteBuffer dst, int srcIdx, int dstIdx, int cnt,
+    @Override public final void copyItems(long srcPageAddr, long dstPageAddr, int srcIdx, int dstIdx, int cnt,
         boolean cpLeft) throws IgniteCheckedException {
-        assert srcIdx != dstIdx || src != dst;
+        assert srcIdx != dstIdx || srcPageAddr != dstPageAddr;
 
-        PageHandler.copyMemory(src, dst, offset(srcIdx), offset(dstIdx), cnt * itemSize);
+        PageHandler.copyMemory(srcPageAddr, dstPageAddr, offset(srcIdx), offset(dstIdx), cnt * itemSize);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
index 15a49ef..5925547 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
 /**
  * IO routines for B+Tree meta pages.
@@ -37,45 +37,48 @@ public class BPlusMetaIO extends PageIO {
     /**
      * @param ver Page format version.
      */
-    protected BPlusMetaIO(int ver) {
+    private BPlusMetaIO(int ver) {
         super(T_BPLUS_META, ver);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAdrr Page address.
      * @param rootId Root page ID.
+     * @param pageSize Page size.
      */
-    public void initRoot(ByteBuffer buf, long rootId) {
-        setLevelsCount(buf, 1);
-        setFirstPageId(buf, 0, rootId);
+    public void initRoot(long pageAdrr, long rootId, int pageSize) {
+        setLevelsCount(pageAdrr, 1, pageSize);
+        setFirstPageId(pageAdrr, 0, rootId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of levels in this tree.
      */
-    public int getLevelsCount(ByteBuffer buf) {
-        return buf.get(LVLS_OFF);
+    public int getLevelsCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, LVLS_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return Max levels possible for this page size.
      */
-    public int getMaxLevels(ByteBuffer buf) {
-        return (buf.capacity() - REFS_OFF) / 8;
+    private int getMaxLevels(long pageAddr, int pageSize) {
+        return (pageSize - REFS_OFF) / 8;
     }
 
     /**
-     * @param buf  Buffer.
+     * @param pageAddr Page address.
      * @param lvls Number of levels in this tree.
+     * @param pageSize Page size.
      */
-    public void setLevelsCount(ByteBuffer buf, int lvls) {
-        assert lvls >= 0 && lvls <= getMaxLevels(buf): lvls;
+    private void setLevelsCount(long pageAddr, int lvls, int pageSize) {
+        assert lvls >= 0 && lvls <= getMaxLevels(pageAddr, pageSize) : lvls;
 
-        buf.put(LVLS_OFF, (byte)lvls);
+        PageUtils.putByte(pageAddr, LVLS_OFF, (byte)lvls);
 
-        assert getLevelsCount(buf) == lvls;
+        assert getLevelsCount(pageAddr) == lvls;
     }
 
     /**
@@ -87,33 +90,33 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lvl Level.
      * @return First page ID at that level.
      */
-    public long getFirstPageId(ByteBuffer buf, int lvl) {
-        return buf.getLong(offset(lvl));
+    public long getFirstPageId(long pageAddr, int lvl) {
+        return PageUtils.getLong(pageAddr, offset(lvl));
     }
 
     /**
-     * @param buf    Buffer.
+     * @param pageAddr Page address.
      * @param lvl    Level.
      * @param pageId Page ID.
      */
-    public void setFirstPageId(ByteBuffer buf, int lvl, long pageId) {
-        assert lvl >= 0 && lvl < getLevelsCount(buf);
+    private void setFirstPageId(long pageAddr, int lvl, long pageId) {
+        assert lvl >= 0 && lvl < getLevelsCount(pageAddr);
 
-        buf.putLong(offset(lvl), pageId);
+        PageUtils.putLong(pageAddr, offset(lvl), pageId);
 
-        assert getFirstPageId(buf, lvl) == pageId;
+        assert getFirstPageId(pageAddr, lvl) == pageId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Root level.
      */
-    public int getRootLevel(ByteBuffer buf) {
-        int lvls = getLevelsCount(buf); // The highest level page is root.
+    public int getRootLevel(long pageAddr) {
+        int lvls = getLevelsCount(pageAddr); // The highest level page is root.
 
         assert lvls > 0 : lvls;
 
@@ -121,22 +124,24 @@ public class BPlusMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rootPageId New root page ID.
+     * @param pageSize Page size.
      */
-    public void addRoot(ByteBuffer buf, long rootPageId) {
-        int lvl = getLevelsCount(buf);
+    public void addRoot(long pageAddr, long rootPageId, int pageSize) {
+        int lvl = getLevelsCount(pageAddr);
 
-        setLevelsCount(buf, lvl + 1);
-        setFirstPageId(buf, lvl, rootPageId);
+        setLevelsCount(pageAddr, lvl + 1, pageSize);
+        setFirstPageId(pageAddr, lvl, rootPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      */
-    public void cutRoot(ByteBuffer buf) {
-        int lvl = getRootLevel(buf);
+    public void cutRoot(long pageAddr, int pageSize) {
+        int lvl = getRootLevel(pageAddr);
 
-        setLevelsCount(buf, lvl); // Decrease tree height.
+        setLevelsCount(pageAddr, lvl, pageSize); // Decrease tree height.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
index 5e9fd6d..8a630cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/CacheVersionIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
@@ -77,6 +78,29 @@ public class CacheVersionIO {
     }
 
     /**
+     * @param addr Write address.
+     * @param ver Version to write.
+     * @param allowNull Is {@code null} version allowed.
+     */
+    public static void write(long addr, GridCacheVersion ver, boolean allowNull) {
+        if (ver == null) {
+            if (allowNull)
+                PageUtils.putByte(addr, 0, NULL_PROTO_VER);
+            else
+                throw new IllegalStateException("Cache version is null");
+        }
+        else {
+            byte protoVer = 1; // Version of serialization protocol.
+
+            PageUtils.putByte(addr, 0, protoVer);
+            PageUtils.putInt(addr, 1, ver.topologyVersion());
+            PageUtils.putInt(addr, 5, ver.nodeOrderAndDrIdRaw());
+            PageUtils.putLong(addr, 9, ver.globalTime());
+            PageUtils.putLong(addr, 17, ver.order());
+        }
+    }
+
+    /**
      * @param protoVer Serialization protocol version.
      * @param allowNull Is {@code null} version allowed.
      * @return Protocol version.
@@ -139,4 +163,26 @@ public class CacheVersionIO {
 
         return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
     }
+
+    /**
+     * Reads GridCacheVersion instance from the given address.
+     *
+     * @param pageAddr Page address.
+     * @param allowNull Is {@code null} version allowed.
+     * @return Version.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static GridCacheVersion read(long pageAddr, boolean allowNull) throws IgniteCheckedException {
+        byte protoVer = checkProtocolVersion(PageUtils.getByte(pageAddr, 0), allowNull);
+
+        if (protoVer == NULL_PROTO_VER)
+            return null;
+
+        int topVer = PageUtils.getInt(pageAddr, 1);
+        int nodeOrderDrId = PageUtils.getInt(pageAddr, 5);
+        long globalTime = PageUtils.getLong(pageAddr, 9);
+        long order = PageUtils.getLong(pageAddr, 17);
+
+        return new GridCacheVersion(topVer, nodeOrderDrId, globalTime, order);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
index a69caab..548e300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPageIO.java
@@ -23,6 +23,8 @@ import java.util.Collection;
 import java.util.HashSet;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
@@ -88,47 +90,48 @@ public class DataPageIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmptyPage(buf);
-        setFreeListPageId(buf, 0L);
+        setEmptyPage(pageAddr, pageSize);
+        setFreeListPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      */
-    private void setEmptyPage(ByteBuffer buf) {
-        setDirectCount(buf, 0);
-        setIndirectCount(buf, 0);
-        setFirstEntryOffset(buf, buf.capacity());
-        setRealFreeSpace(buf, buf.capacity() - ITEMS_OFF);
+    private void setEmptyPage(long pageAddr, int pageSize) {
+        setDirectCount(pageAddr, 0);
+        setIndirectCount(pageAddr, 0);
+        setFirstEntryOffset(pageAddr, pageSize, pageSize);
+        setRealFreeSpace(pageAddr, pageSize - ITEMS_OFF, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param freeListPageId Free list page ID.
      */
-    public void setFreeListPageId(ByteBuffer buf, long freeListPageId) {
-        buf.putLong(FREE_LIST_PAGE_ID_OFF, freeListPageId);
+    public void setFreeListPageId(long pageAddr, long freeListPageId) {
+        PageUtils.putLong(pageAddr, FREE_LIST_PAGE_ID_OFF, freeListPageId);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @return Free list page ID.
      */
-    public long getFreeListPageId(ByteBuffer buf) {
-        return buf.getLong(FREE_LIST_PAGE_ID_OFF);
+    public long getFreeListPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, FREE_LIST_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param show What elements of data page entry to show in the result.
      * @return Data page entry size.
      */
-    private int getPageEntrySize(ByteBuffer buf, int dataOff, int show) {
-        int payloadLen = buf.getShort(dataOff) & 0xFFFF;
+    private int getPageEntrySize(long pageAddr, int dataOff, int show) {
+        int payloadLen = PageUtils.getShort(pageAddr, dataOff) & 0xFFFF;
 
         if ((payloadLen & FRAGMENTED_FLAG) != 0)
             payloadLen &= ~FRAGMENTED_FLAG; // We are fragmented and have a link.
@@ -144,7 +147,7 @@ public class DataPageIO extends PageIO {
      * @return Data page entry size.
      */
     private int getPageEntrySize(int payloadLen, int show) {
-        assert payloadLen > 0: payloadLen;
+        assert payloadLen > 0 : payloadLen;
 
         int res = payloadLen;
 
@@ -161,45 +164,47 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Entry data offset.
+     * @param pageSize Page size.
      */
-    private void setFirstEntryOffset(ByteBuffer buf, int dataOff) {
-        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= buf.capacity(): dataOff;
+    private void setFirstEntryOffset(long pageAddr, int dataOff, int pageSize) {
+        assert dataOff >= ITEMS_OFF + ITEM_SIZE && dataOff <= pageSize : dataOff;
 
-        buf.putShort(FIRST_ENTRY_OFF, (short)dataOff);
+        PageUtils.putShort(pageAddr, FIRST_ENTRY_OFF, (short)dataOff);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Entry data offset.
      */
-    private int getFirstEntryOffset(ByteBuffer buf) {
-        return buf.getShort(FIRST_ENTRY_OFF) & 0xFFFF;
+    private int getFirstEntryOffset(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FIRST_ENTRY_OFF) & 0xFFFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeSpace Free space.
+     * @param pageSize Page size.
      */
-    private void setRealFreeSpace(ByteBuffer buf, int freeSpace) {
-        assert freeSpace == actualFreeSpace(buf): freeSpace + " != " + actualFreeSpace(buf);
+    private void setRealFreeSpace(long pageAddr, int freeSpace, int pageSize) {
+        assert freeSpace == actualFreeSpace(pageAddr, pageSize) : freeSpace + " != " + actualFreeSpace(pageAddr, pageSize);
 
-        buf.putShort(FREE_SPACE_OFF, (short)freeSpace);
+        PageUtils.putShort(pageAddr, FREE_SPACE_OFF, (short)freeSpace);
     }
 
     /**
      * Free space refers to a "max row size (without any data page specific overhead) which is
      * guaranteed to fit into this data page".
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    public int getFreeSpace(ByteBuffer buf) {
-        if (getFreeItemSlots(buf) == 0)
+    public int getFreeSpace(long pageAddr) {
+        if (getFreeItemSlots(pageAddr) == 0)
             return 0;
 
-        int freeSpace = getRealFreeSpace(buf);
+        int freeSpace = getRealFreeSpace(pageAddr);
 
         // We reserve size here because of getFreeSpace() method semantics (see method javadoc).
         // It means that we must be able to accommodate a row of size which is equal to getFreeSpace(),
@@ -211,48 +216,49 @@ public class DataPageIO extends PageIO {
     }
 
     /**
+     * @param pageAddr Page address.
      * @return {@code true} If there is no useful data in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
-        return getDirectCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getDirectCount(pageAddr) == 0;
     }
 
     /**
-     * Equivalent for {@link #actualFreeSpace(ByteBuffer)} but reads saved value.
+     * Equivalent for {@link #actualFreeSpace(long, int)} but reads saved value.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Free space.
      */
-    private int getRealFreeSpace(ByteBuffer buf) {
-        return buf.getShort(FREE_SPACE_OFF);
+    private int getRealFreeSpace(long pageAddr) {
+        return PageUtils.getShort(pageAddr, FREE_SPACE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Direct count.
      */
-    private void setDirectCount(ByteBuffer buf, int cnt) {
+    private void setDirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(DIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, DIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Direct count.
      */
-    private int getDirectCount(ByteBuffer buf) {
-        return buf.get(DIRECT_CNT_OFF) & 0xFF;
+    private int getDirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, DIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cnt Indirect count.
      */
-    private void setIndirectCount(ByteBuffer buf, int cnt) {
+    private void setIndirectCount(long pageAddr, int cnt) {
         assert checkCount(cnt): cnt;
 
-        buf.put(INDIRECT_CNT_OFF, (byte)cnt);
+        PageUtils.putByte(pageAddr, INDIRECT_CNT_OFF, (byte)cnt);
     }
 
     /**
@@ -272,36 +278,36 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Indirect count.
      */
-    private int getIndirectCount(ByteBuffer buf) {
-        return buf.get(INDIRECT_CNT_OFF) & 0xFF;
+    private int getIndirectCount(long pageAddr) {
+        return PageUtils.getByte(pageAddr, INDIRECT_CNT_OFF) & 0xFF;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Number of free entry slots.
      */
-    private int getFreeItemSlots(ByteBuffer buf) {
-        return 0xFF - getDirectCount(buf);
+    private int getFreeItemSlots(long pageAddr) {
+        return 0xFF - getDirectCount(pageAddr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Found index of indirect item.
      */
-    private int findIndirectItemIndex(ByteBuffer buf, int itemId, int directCnt, int indirectCnt) {
+    private int findIndirectItemIndex(long pageAddr, int itemId, int directCnt, int indirectCnt) {
         int low = directCnt;
         int high = directCnt + indirectCnt - 1;
 
         while (low <= high) {
             int mid = (low + high) >>> 1;
 
-            int cmp = Integer.compare(itemId(getItem(buf, mid)), itemId);
+            int cmp = Integer.compare(itemId(getItem(pageAddr, mid)), itemId);
 
             if (cmp < 0)
                 low = mid + 1;
@@ -315,19 +321,20 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param pageSize Page size.
      * @return String representation.
      */
-    public String printPageLayout(ByteBuffer buf) {
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
-        int free = getRealFreeSpace(buf);
+    private String printPageLayout(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
+        int free = getRealFreeSpace(pageAddr);
 
         boolean valid = directCnt >= indirectCnt;
 
         SB b = new SB();
 
-        b.appendHex(PageIO.getPageId(buf)).a(" [");
+        b.appendHex(PageIO.getPageId(pageAddr)).a(" [");
 
         int entriesSize = 0;
 
@@ -335,12 +342,12 @@ public class DataPageIO extends PageIO {
             if (i != 0)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
-            if (item < ITEMS_OFF || item >= buf.capacity())
+            if (item < ITEMS_OFF || item >= pageSize)
                 valid = false;
 
-            entriesSize += getPageEntrySize(buf, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            entriesSize += getPageEntrySize(pageAddr, item, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             b.a(item);
         }
@@ -353,7 +360,7 @@ public class DataPageIO extends PageIO {
             if (i != directCnt)
                 b.a(", ");
 
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             int itemId = itemId(item);
             int directIdx = directItemIndex(item);
@@ -366,7 +373,7 @@ public class DataPageIO extends PageIO {
             if (itemId < directCnt || directIdx < 0 || directIdx >= directCnt)
                 valid = false;
 
-            if (i > directCnt && itemId(getItem(buf, i - 1)) >= itemId)
+            if (i > directCnt && itemId(getItem(pageAddr, i - 1)) >= itemId)
                 valid = false;
 
 
@@ -375,7 +382,7 @@ public class DataPageIO extends PageIO {
 
         b.a("][free=").a(free);
 
-        int actualFree = buf.capacity() - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
+        int actualFree = pageSize - ITEMS_OFF - (entriesSize + (directCnt + indirectCnt) * ITEM_SIZE);
 
         if (free != actualFree) {
             b.a(", actualFree=").a(actualFree);
@@ -391,94 +398,94 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Data entry offset in bytes.
      */
-    private int getDataOffset(ByteBuffer buf, int itemId) {
+    private int getDataOffset(long pageAddr, int itemId, int pageSize) {
         assert checkIndex(itemId): itemId;
 
-        int directCnt = getDirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
 
-        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(buf);
+        assert directCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", page=" + printPageLayout(pageAddr, pageSize);
 
         if (itemId >= directCnt) { // Need to do indirect lookup.
-            int indirectCnt = getIndirectCount(buf);
+            int indirectCnt = getIndirectCount(pageAddr);
 
             // Must have indirect items here.
             assert indirectCnt > 0: "itemId=" + itemId + ", directCnt=" + directCnt + ", indirectCnt=" + indirectCnt +
-                ", page=" + printPageLayout(buf);
+                ", page=" + printPageLayout(pageAddr, pageSize);
 
-            int indirectItemIdx = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+            int indirectItemIdx = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
             assert indirectItemIdx >= directCnt : indirectItemIdx + " " + directCnt;
             assert indirectItemIdx < directCnt + indirectCnt: indirectItemIdx + " " + directCnt + " " + indirectCnt;
 
-            itemId = directItemIndex(getItem(buf, indirectItemIdx));
+            itemId = directItemIndex(getItem(pageAddr, indirectItemIdx));
 
             assert itemId >= 0 && itemId < directCnt: itemId + " " + directCnt + " " + indirectCnt; // Direct item.
         }
 
-        return directItemToOffset(getItem(buf, itemId));
+        return directItemToOffset(getItem(pageAddr, itemId));
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param dataOff Points to the entry start.
      * @return Link to the next entry fragment or 0 if no fragments left or if entry is not fragmented.
      */
-    private long getNextFragmentLink(ByteBuffer buf, int dataOff) {
-        assert isFragmented(buf, dataOff);
+    private long getNextFragmentLink(long pageAddr, int dataOff) {
+        assert isFragmented(pageAddr, dataOff);
 
-        return buf.getLong(dataOff + PAYLOAD_LEN_SIZE);
+        return PageUtils.getLong(pageAddr, dataOff + PAYLOAD_LEN_SIZE);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @return {@code true} If the data row is fragmented across multiple pages.
      */
-    private boolean isFragmented(ByteBuffer buf, int dataOff) {
-        return (buf.getShort(dataOff) & FRAGMENTED_FLAG) != 0;
+    private boolean isFragmented(long pageAddr, int dataOff) {
+        return (PageUtils.getShort(pageAddr, dataOff) & FRAGMENTED_FLAG) != 0;
     }
 
     /**
      * Sets position to start of actual fragment data and limit to it's end.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param itemId Item to position on.
+     * @param pageSize Page size.
      * @return Link to the next fragment or {@code 0} if it is the last fragment or the data row is not fragmented.
      */
-    public long setPositionAndLimitOnPayload(final ByteBuffer buf, final int itemId) {
-        int dataOff = getDataOffset(buf, itemId);
-
-        boolean fragmented = isFragmented(buf, dataOff);
-        long nextLink = fragmented ? getNextFragmentLink(buf, dataOff) : 0;
-        int payloadSize = getPageEntrySize(buf, dataOff, 0);
+    public DataPagePayload readPayload(final long pageAddr, final int itemId, final int pageSize) {
+        int dataOff = getDataOffset(pageAddr, itemId, pageSize);
 
-        buf.position(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0));
+        boolean fragmented = isFragmented(pageAddr, dataOff);
+        long nextLink = fragmented ? getNextFragmentLink(pageAddr, dataOff) : 0;
+        int payloadSize = getPageEntrySize(pageAddr, dataOff, 0);
 
-        buf.limit(buf.position() + payloadSize);
-
-        return nextLink;
+        return new DataPagePayload(dataOff + PAYLOAD_LEN_SIZE + (fragmented ? LINK_SIZE : 0),
+            payloadSize,
+            nextLink);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item.
      */
-    private short getItem(ByteBuffer buf, int idx) {
-        return buf.getShort(itemOffset(idx));
+    private short getItem(long pageAddr, int idx) {
+        return PageUtils.getShort(pageAddr, itemOffset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param item Item.
      */
-    private void setItem(ByteBuffer buf, int idx, short item) {
-        buf.putShort(itemOffset(idx), item);
+    private void setItem(long pageAddr, int idx, short item) {
+        PageUtils.putShort(pageAddr, itemOffset(idx), item);
     }
 
     /**
@@ -540,14 +547,14 @@ public class DataPageIO extends PageIO {
     /**
      * Move the last direct item to the free slot and reference it with indirect item on the same place.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param freeDirectIdx Free slot.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return {@code true} If the last direct item already had corresponding indirect item.
      */
-    private boolean moveLastItem(ByteBuffer buf, int freeDirectIdx, int directCnt, int indirectCnt) {
-        int lastIndirectId = findIndirectIndexForLastDirect(buf, directCnt, indirectCnt);
+    private boolean moveLastItem(long pageAddr, int freeDirectIdx, int directCnt, int indirectCnt) {
+        int lastIndirectId = findIndirectIndexForLastDirect(pageAddr, directCnt, indirectCnt);
 
         int lastItemId = directCnt - 1;
 
@@ -557,13 +564,13 @@ public class DataPageIO extends PageIO {
 
         assert itemId(indirectItem) == lastItemId && directItemIndex(indirectItem) == freeDirectIdx;
 
-        setItem(buf, freeDirectIdx, getItem(buf, lastItemId));
-        setItem(buf, lastItemId, indirectItem);
+        setItem(pageAddr, freeDirectIdx, getItem(pageAddr, lastItemId));
+        setItem(pageAddr, lastItemId, indirectItem);
 
-        assert getItem(buf, lastItemId) == indirectItem;
+        assert getItem(pageAddr, lastItemId) == indirectItem;
 
         if (lastIndirectId != -1) { // Fix pointer to direct item.
-            setItem(buf, lastIndirectId, indirectItem(itemId(getItem(buf, lastIndirectId)), freeDirectIdx));
+            setItem(pageAddr, lastIndirectId, indirectItem(itemId(getItem(pageAddr, lastIndirectId)), freeDirectIdx));
 
             return true;
         }
@@ -572,16 +579,16 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @return Index of indirect item for the last direct item.
      */
-    private int findIndirectIndexForLastDirect(ByteBuffer buf, int directCnt, int indirectCnt) {
+    private int findIndirectIndexForLastDirect(long pageAddr, int directCnt, int indirectCnt) {
         int lastDirectId = directCnt - 1;
 
         for (int i = directCnt, end = directCnt + indirectCnt; i < end; i++) {
-            short item = getItem(buf, i);
+            short item = getItem(pageAddr, i);
 
             if (directItemIndex(item) == lastDirectId)
                 return i;
@@ -591,20 +598,21 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param itemId Fixed item ID (the index used for referencing an entry from the outside).
+     * @param pageSize Page size.
      * @return Next link for fragmented entries or {@code 0} if none.
      * @throws IgniteCheckedException If failed.
      */
-    public long removeRow(ByteBuffer buf, int itemId) throws IgniteCheckedException {
+    public long removeRow(long pageAddr, int itemId, int pageSize) throws IgniteCheckedException {
         assert checkIndex(itemId) : itemId;
 
-        final int dataOff = getDataOffset(buf, itemId);
-        final long nextLink = isFragmented(buf, dataOff) ? getNextFragmentLink(buf, dataOff) : 0;
+        final int dataOff = getDataOffset(pageAddr, itemId, pageSize);
+        final long nextLink = isFragmented(pageAddr, dataOff) ? getNextFragmentLink(pageAddr, dataOff) : 0;
 
         // Record original counts to calculate delta in free space in the end of remove.
-        final int directCnt = getDirectCount(buf);
-        final int indirectCnt = getIndirectCount(buf);
+        final int directCnt = getDirectCount(pageAddr);
+        final int indirectCnt = getIndirectCount(pageAddr);
 
         int curIndirectCnt = indirectCnt;
 
@@ -613,24 +621,24 @@ public class DataPageIO extends PageIO {
         // Remove the last item on the page.
         if (directCnt == 1) {
             assert (indirectCnt == 0 && itemId == 0) ||
-                (indirectCnt == 1 && itemId == itemId(getItem(buf, 1))) : itemId;
+                (indirectCnt == 1 && itemId == itemId(getItem(pageAddr, 1))) : itemId;
 
-            setEmptyPage(buf);
+            setEmptyPage(pageAddr, pageSize);
         }
         else {
             // Get the entry size before the actual remove.
-            int rmvEntrySize = getPageEntrySize(buf, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int rmvEntrySize = getPageEntrySize(pageAddr, dataOff, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int indirectId = 0;
 
             if (itemId >= directCnt) { // Need to remove indirect item.
                 assert indirectCnt > 0;
 
-                indirectId = findIndirectItemIndex(buf, itemId, directCnt, indirectCnt);
+                indirectId = findIndirectItemIndex(pageAddr, itemId, directCnt, indirectCnt);
 
                 assert indirectId >= directCnt;
 
-                itemId = directItemIndex(getItem(buf, indirectId));
+                itemId = directItemIndex(getItem(pageAddr, indirectId));
 
                 assert itemId < directCnt;
             }
@@ -638,48 +646,50 @@ public class DataPageIO extends PageIO {
             boolean dropLast = true;
 
             if (itemId + 1 < directCnt) // It is not the last direct item.
-                dropLast = moveLastItem(buf, itemId, directCnt, indirectCnt);
+                dropLast = moveLastItem(pageAddr, itemId, directCnt, indirectCnt);
 
             if (indirectId == 0) {// For the last direct item with no indirect item.
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectCnt, -1);
+                    moveItems(pageAddr, directCnt, indirectCnt, -1, pageSize);
                 else
                     curIndirectCnt++;
             }
             else {
                 if (dropLast)
-                    moveItems(buf, directCnt, indirectId - directCnt, -1);
+                    moveItems(pageAddr, directCnt, indirectId - directCnt, -1, pageSize);
 
-                moveItems(buf, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1);
+                moveItems(pageAddr, indirectId + 1, directCnt + indirectCnt - indirectId - 1, dropLast ? -2 : -1, pageSize);
 
                 if (dropLast)
                     curIndirectCnt--;
             }
 
-            setIndirectCount(buf, curIndirectCnt);
-            setDirectCount(buf, directCnt - 1);
+            setIndirectCount(pageAddr, curIndirectCnt);
+            setDirectCount(pageAddr, directCnt - 1);
 
-            assert getIndirectCount(buf) <= getDirectCount(buf);
+            assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
             // Increase free space.
-            setRealFreeSpace(buf, getRealFreeSpace(buf) + rmvEntrySize +
-                ITEM_SIZE * (directCnt - getDirectCount(buf) + indirectCnt - getIndirectCount(buf)));
+            setRealFreeSpace(pageAddr,
+                getRealFreeSpace(pageAddr) + rmvEntrySize + ITEM_SIZE * (directCnt - getDirectCount(pageAddr) + indirectCnt - getIndirectCount(pageAddr)),
+                pageSize);
         }
 
         return nextLink;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveItems(ByteBuffer buf, int idx, int cnt, int step) {
+    private void moveItems(long pageAddr, int idx, int cnt, int step, int pageSize) {
         assert cnt >= 0: cnt;
 
         if (cnt != 0)
-            moveBytes(buf, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE);
+            moveBytes(pageAddr, itemOffset(idx), cnt * ITEM_SIZE, step * ITEM_SIZE, pageSize);
     }
 
     /**
@@ -696,73 +706,80 @@ public class DataPageIO extends PageIO {
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
+        final long pageAddr,
         CacheDataRow row,
-        int rowSize
+        final int rowSize,
+        final int pageSize
     ) throws IgniteCheckedException {
-        assert rowSize <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert rowSize <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(rowSize, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, rowSize, row);
+        writeRowData(pageAddr, dataOff, rowSize, row);
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
-        setLink(row, buf, itemId);
+        setLink(row, pageAddr, itemId);
     }
 
     /**
      * Adds row to this data page and sets respective link to the given row object.
      *
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @param payload Payload.
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRow(
-        ByteBuffer buf,
-        byte[] payload
+        long pageAddr,
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
-        assert payload.length <= getFreeSpace(buf): "can't call addRow if not enough space for the whole row";
+        assert payload.length <= getFreeSpace(pageAddr): "can't call addRow if not enough space for the whole row";
 
         int fullEntrySize = getPageEntrySize(payload.length, SHOW_PAYLOAD_LEN | SHOW_ITEM);
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        writeRowData(buf, dataOff, payload);
+        writeRowData(pageAddr, dataOff, payload);
 
-        addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
     }
 
     /**
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param entryFullSize New entry full size (with item, length and link).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff First entry offset.
+     * @param pageSize Page size.
      * @return First entry offset after compaction.
      */
     private int compactIfNeed(
-        final ByteBuffer buf,
+        final long pageAddr,
         final int entryFullSize,
         final int directCnt,
         final int indirectCnt,
-        int dataOff
+        int dataOff,
+        int pageSize
     ) {
         if (!isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt)) {
-            dataOff = compactDataEntries(buf, directCnt);
+            dataOff = compactDataEntries(pageAddr, directCnt, pageSize);
 
             assert isEnoughSpace(entryFullSize, dataOff, directCnt, indirectCnt);
         }
@@ -773,41 +790,49 @@ public class DataPageIO extends PageIO {
     /**
      * Put item reference on entry.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size (with link, payload size and item).
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
      * @param dataOff Data offset.
+     * @param pageSize Page size.
      * @return Item ID.
      */
-    private int addItem(final ByteBuffer buf, final int fullEntrySize, final int directCnt,
-        final int indirectCnt, final int dataOff) {
-        setFirstEntryOffset(buf, dataOff);
+    private int addItem(final long pageAddr,
+        final int fullEntrySize,
+        final int directCnt,
+        final int indirectCnt,
+        final int dataOff,
+        final int pageSize)
+    {
+        setFirstEntryOffset(pageAddr, dataOff, pageSize);
 
-        int itemId = insertItem(buf, dataOff, directCnt, indirectCnt);
+        int itemId = insertItem(pageAddr, dataOff, directCnt, indirectCnt, pageSize);
 
         assert checkIndex(itemId): itemId;
-        assert getIndirectCount(buf) <= getDirectCount(buf);
+        assert getIndirectCount(pageAddr) <= getDirectCount(pageAddr);
 
         // Update free space. If number of indirect items changed, then we were able to reuse an item slot.
-        setRealFreeSpace(buf, getRealFreeSpace(buf) - fullEntrySize +
-            (getIndirectCount(buf) != indirectCnt ? ITEM_SIZE : 0));
+        setRealFreeSpace(pageAddr,
+            getRealFreeSpace(pageAddr) - fullEntrySize + (getIndirectCount(pageAddr) != indirectCnt ? ITEM_SIZE : 0),
+            pageSize);
 
         return itemId;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param fullEntrySize Full entry size.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Offset in the buffer where the entry must be written.
      */
-    private int getDataOffsetForWrite(ByteBuffer buf, int fullEntrySize, int directCnt, int indirectCnt) {
-        int dataOff = getFirstEntryOffset(buf);
+    private int getDataOffsetForWrite(long pageAddr, int fullEntrySize, int directCnt, int indirectCnt, int pageSize) {
+        int dataOff = getFirstEntryOffset(pageAddr);
 
         // Compact if we do not have enough space for entry.
-        dataOff = compactIfNeed(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        dataOff = compactIfNeed(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         // We will write data right before the first entry.
         dataOff -= fullEntrySize - ITEM_SIZE;
@@ -818,91 +843,105 @@ public class DataPageIO extends PageIO {
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @param row Cache data row.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
     public int addRowFragment(
-        ByteBuffer buf,
+        PageMemory pageMem,
+        long pageAddr,
         CacheDataRow row,
         int written,
-        int rowSize
+        int rowSize,
+        int pageSize
     ) throws IgniteCheckedException {
-        return addRowFragment(buf, written, rowSize, row.link(), row, null);
+        return addRowFragment(pageMem, pageAddr, written, rowSize, row.link(), row, null, pageSize);
     }
 
     /**
      * Adds this payload as a fragment to this data page.
      *
-     * @param buf Byte buffer.
+     * @param pageAddr Page address.
      * @param payload Payload bytes.
      * @param lastLink Link to the previous written fragment (link to the tail).
+     * @param pageSize Page size.
      * @throws IgniteCheckedException If failed.
      */
     public void addRowFragment(
-        ByteBuffer buf,
+        long pageAddr,
         byte[] payload,
-        long lastLink
+        long lastLink,
+        int pageSize
     ) throws IgniteCheckedException {
-        addRowFragment(buf, 0, 0, lastLink, null, payload);
+        addRowFragment(null, pageAddr, 0, 0, lastLink, null, payload, pageSize);
     }
 
     /**
      * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
      *
-     * @param buf Byte buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @param written Number of bytes of row size that was already written.
      * @param rowSize Row size.
      * @param lastLink Link to the previous written fragment (link to the tail).
      * @param row Row.
      * @param payload Payload bytes.
+     * @param pageSize Page size.
      * @return Written payload size.
      * @throws IgniteCheckedException If failed.
      */
     private int addRowFragment(
-        ByteBuffer buf,
+        PageMemory pageMem,
+        long pageAddr,
         int written,
         int rowSize,
         long lastLink,
         CacheDataRow row,
-        byte[] payload
+        byte[] payload,
+        int pageSize
     ) throws IgniteCheckedException {
         assert payload == null ^ row == null;
 
-        int directCnt = getDirectCount(buf);
-        int indirectCnt = getIndirectCount(buf);
+        int directCnt = getDirectCount(pageAddr);
+        int indirectCnt = getIndirectCount(pageAddr);
 
         int payloadSize = payload != null ? payload.length :
-            Math.min(rowSize - written, getFreeSpace(buf));
+            Math.min(rowSize - written, getFreeSpace(pageAddr));
 
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
-        int dataOff = getDataOffsetForWrite(buf, fullEntrySize, directCnt, indirectCnt);
+        int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
+
+        if (payload == null) {
+            ByteBuffer buf = pageMem.pageBuffer(pageAddr);
 
-        try {
             buf.position(dataOff);
 
-            buf.putShort((short) (payloadSize | FRAGMENTED_FLAG));
+            short p = (short)(payloadSize | FRAGMENTED_FLAG);
+
+            buf.putShort(p);
             buf.putLong(lastLink);
 
-            if (payload == null) {
-                int rowOff = rowSize - written - payloadSize;
+            int rowOff = rowSize - written - payloadSize;
 
-                writeFragmentData(row, buf, rowOff, payloadSize);
-            }
-            else
-                buf.put(payload);
+            writeFragmentData(row, buf, rowOff, payloadSize);
         }
-        finally {
-            buf.position(0);
+        else {
+            PageUtils.putShort(pageAddr, dataOff, (short)(payloadSize | FRAGMENTED_FLAG));
+
+            PageUtils.putLong(pageAddr, dataOff + 2, lastLink);
+
+            PageUtils.putBytes(pageAddr, dataOff + 10, payload);
         }
 
-        int itemId = addItem(buf, fullEntrySize, directCnt, indirectCnt, dataOff);
+        int itemId = addItem(pageAddr, fullEntrySize, directCnt, indirectCnt, dataOff, pageSize);
 
         if (row != null)
-            setLink(row, buf, itemId);
+            setLink(row, pageAddr, itemId);
 
         return payloadSize;
     }
@@ -912,7 +951,7 @@ public class DataPageIO extends PageIO {
      * @param buf Page buffer.
      * @param itemId Item ID.
      */
-    private void setLink(CacheDataRow row, ByteBuffer buf, int itemId) {
+    private void setLink(CacheDataRow row, long buf, int itemId) {
         row.link(PageIdUtils.link(getPageId(buf), itemId));
     }
 
@@ -1083,54 +1122,56 @@ public class DataPageIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param directCnt Direct items count.
      * @param indirectCnt Indirect items count.
+     * @param pageSize Page size.
      * @return Item ID (insertion index).
      */
-    private int insertItem(ByteBuffer buf, int dataOff, int directCnt, int indirectCnt) {
+    private int insertItem(long pageAddr, int dataOff, int directCnt, int indirectCnt, int pageSize) {
         if (indirectCnt > 0) {
             // If the first indirect item is on correct place to become the last direct item, do the transition
             // and insert the new item into the free slot which was referenced by this first indirect item.
-            short item = getItem(buf, directCnt);
+            short item = getItem(pageAddr, directCnt);
 
             if (itemId(item) == directCnt) {
                 int directItemIdx = directItemIndex(item);
 
-                setItem(buf, directCnt, getItem(buf, directItemIdx));
-                setItem(buf, directItemIdx, directItemFromOffset(dataOff));
+                setItem(pageAddr, directCnt, getItem(pageAddr, directItemIdx));
+                setItem(pageAddr, directItemIdx, directItemFromOffset(dataOff));
 
-                setDirectCount(buf, directCnt + 1);
-                setIndirectCount(buf, indirectCnt - 1);
+                setDirectCount(pageAddr, directCnt + 1);
+                setIndirectCount(pageAddr, indirectCnt - 1);
 
                 return directItemIdx;
             }
         }
 
         // Move all the indirect items forward to make a free slot and insert new item at the end of direct items.
-        moveItems(buf, directCnt, indirectCnt, +1);
+        moveItems(pageAddr, directCnt, indirectCnt, +1, pageSize);
 
-        setItem(buf, directCnt, directItemFromOffset(dataOff));
+        setItem(pageAddr, directCnt, directItemFromOffset(dataOff));
 
-        setDirectCount(buf, directCnt + 1);
-        assert getDirectCount(buf) == directCnt + 1;
+        setDirectCount(pageAddr, directCnt + 1);
+        assert getDirectCount(pageAddr) == directCnt + 1;
 
         return directCnt; // Previous directCnt will be our itemId.
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param directCnt Direct items count.
+     * @param pageSize Page size.
      * @return New first entry offset.
      */
-    private int compactDataEntries(ByteBuffer buf, int directCnt) {
+    private int compactDataEntries(long pageAddr, int directCnt, int pageSize) {
         assert checkCount(directCnt): directCnt;
 
         int[] offs = new int[directCnt];
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
             offs[i] = (off << 8) | i; // This way we'll be able to sort by offset using Arrays.sort(...).
         }
@@ -1138,27 +1179,27 @@ public class DataPageIO extends PageIO {
         Arrays.sort(offs);
 
         // Move right all of the entries if possible to make the page as compact as possible to its tail.
-        int prevOff = buf.capacity();
+        int prevOff = pageSize;
 
         for (int i = directCnt - 1; i >= 0; i--) {
             int off = offs[i] >>> 8;
 
             assert off < prevOff: off;
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             int delta = prevOff - (off + entrySize);
 
             if (delta != 0) { // Move right.
                 assert delta > 0: delta;
 
-                moveBytes(buf, off, entrySize, delta);
+                moveBytes(pageAddr, off, entrySize, delta, pageSize);
 
                 int itemId = offs[i] & 0xFF;
 
                 off += delta;
 
-                setItem(buf, itemId, directItemFromOffset(off));
+                setItem(pageAddr, itemId, directItemFromOffset(off));
             }
 
             prevOff = off;
@@ -1170,94 +1211,82 @@ public class DataPageIO extends PageIO {
     /**
      * Full-scan free space calculation procedure.
      *
-     * @param buf Buffer to scan.
+     * @param pageAddr Page to scan.
+     * @param pageSize Page size.
      * @return Actual free space in the buffer.
      */
-    private int actualFreeSpace(ByteBuffer buf) {
-        int directCnt = getDirectCount(buf);
+    private int actualFreeSpace(long pageAddr, int pageSize) {
+        int directCnt = getDirectCount(pageAddr);
 
         int entriesSize = 0;
 
         for (int i = 0; i < directCnt; i++) {
-            int off = directItemToOffset(getItem(buf, i));
+            int off = directItemToOffset(getItem(pageAddr, i));
 
-            int entrySize = getPageEntrySize(buf, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
+            int entrySize = getPageEntrySize(pageAddr, off, SHOW_PAYLOAD_LEN | SHOW_LINK);
 
             entriesSize += entrySize;
         }
 
-        return buf.capacity() - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(buf)) * ITEM_SIZE;
+        return pageSize - ITEMS_OFF - entriesSize - (directCnt + getIndirectCount(pageAddr)) * ITEM_SIZE;
     }
 
     /**
-     * @param buf Buffer.
+     * @param addr Address.
      * @param off Offset.
      * @param cnt Count.
      * @param step Step.
+     * @param pageSize Page size.
      */
-    private void moveBytes(ByteBuffer buf, int off, int cnt, int step) {
+    private void moveBytes(long addr, int off, int cnt, int step, int pageSize) {
         assert step != 0: step;
         assert off + step >= 0;
-        assert off + step + cnt <= buf.capacity() : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
-            ", cap=" + buf.capacity() + ']';
+        assert off + step + cnt <= pageSize : "[off=" + off + ", step=" + step + ", cnt=" + cnt +
+            ", cap=" + pageSize + ']';
 
-        PageHandler.copyMemory(buf, buf, off, off + step, cnt);
+        PageHandler.copyMemory(addr, addr, off, off + step, cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payloadSize Payload size.
      * @param row Data row.
      * @throws IgniteCheckedException If failed.
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long pageAddr,
         int dataOff,
         int payloadSize,
         CacheDataRow row
     ) throws IgniteCheckedException {
-        try {
-            buf.position(dataOff);
+        long addr = pageAddr + dataOff;
 
-            buf.putShort((short)payloadSize);
+        PageUtils.putShort(addr, 0, (short)payloadSize);
+        addr += 2;
 
-            boolean ok = row.key().putValue(buf);
+        addr += row.key().putValue(addr);
+        addr += row.value().putValue(addr);
 
-            assert ok;
+        CacheVersionIO.write(addr, row.version(), false);
+        addr += CacheVersionIO.size(row.version(), false);
 
-            ok = row.value().putValue(buf);
-
-            assert ok;
-
-            CacheVersionIO.write(buf, row.version(), false);
-
-            buf.putLong(row.expireTime());
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putLong(addr, 0, row.expireTime());
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dataOff Data offset.
      * @param payload Payload
      */
     private void writeRowData(
-        ByteBuffer buf,
+        long pageAddr,
         int dataOff,
         byte[] payload
     ) {
-        try {
-            buf.position(dataOff);
-
-            buf.putShort((short)payload.length);
+        PageUtils.putShort(pageAddr, dataOff, (short)payload.length);
+        dataOff += 2;
 
-            buf.put(payload);
-        }
-        finally {
-            buf.position(0);
-        }
+        PageUtils.putBytes(pageAddr, dataOff, payload);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
new file mode 100644
index 0000000..7dedc00
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/DataPagePayload.java
@@ -0,0 +1,64 @@
+/*
+ * 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.database.tree.io;
+
+/**
+ *
+ */
+public class DataPagePayload {
+    /** */
+    private final int off;
+
+    /** */
+    private final int payloadSize;
+
+    /** */
+    private final long nextLink;
+
+    /**
+     * @param off Offset.
+     * @param payloadSize Payload size.
+     * @param nextLink Next link.
+     */
+    DataPagePayload(int off, int payloadSize, long nextLink) {
+        this.off = off;
+        this.payloadSize = payloadSize;
+        this.nextLink = nextLink;
+    }
+
+    /**
+     * @return Offset.
+     */
+    public int offset() {
+        return off;
+    }
+
+    /**
+     * @return Payload size.
+     */
+    public int payloadSize() {
+        return payloadSize;
+    }
+
+    /**
+     * @return Next link.
+     */
+    public long nextLink() {
+        return nextLink;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
index bd1139d..428cb3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/IOVersions.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Registry for IO versions.
  */
@@ -80,16 +78,16 @@ public final class IOVersions<V extends PageIO> {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      */
-    public V forPage(ByteBuffer buf) {
-        int ver = PageIO.getVersion(buf);
+    public V forPage(long pageAddr) {
+        int ver = PageIO.getVersion(pageAddr);
 
         V res = forVersion(ver);
 
-        assert res.getType() == PageIO.getType(buf) : "resType=" + res.getType() +
-            ", pageType=" + PageIO.getType(buf);
+        assert res.getType() == PageIO.getType(pageAddr) : "resType=" + res.getType() +
+            ", pageType=" + PageIO.getType(pageAddr);
 
         return res;
     }


[37/50] [abbrv] ignite git commit: IGNITE-4045 .NET: Support DML API - fix code analysis warnings

Posted by sb...@apache.org.
IGNITE-4045 .NET: Support DML API - fix code analysis warnings


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 7b711a3248b0dafdcea195766b50be4d44656f7f
Parents: b7908d7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jan 16 19:50:35 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 16 19:50:35 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../Binary/BinaryArrayEqualityComparer.cs       |  2 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |  2 +-
 .../Impl/Binary/DateTimeHolder.cs               | 22 +++++++++++-
 .../Impl/Binary/Io/IBinaryStream.cs             | 16 ---------
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    | 36 ++++++++++++++++++++
 6 files changed, 60 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 42ccdd4..d58c872 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -185,6 +185,7 @@
     <Compile Include="Common\IgniteGuid.cs" />
     <Compile Include="Common\Package-Info.cs" />
     <Compile Include="Impl\Binary\DateTimeSerializer.cs" />
+    <Compile Include="Impl\Binary\IO\IBinaryStreamProcessor.cs" />
     <Compile Include="Impl\Binary\SerializableSerializer.cs" />
     <Compile Include="Impl\Binary\BinaryWriterExtensions.cs" />
     <Compile Include="Impl\Cache\Affinity\AffinityFunctionBase.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
index 09f7f0f..4e29a91 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
@@ -26,7 +26,7 @@ namespace Apache.Ignite.Core.Binary
     /// <summary>
     /// Compares binary object equality using underlying byte array.
     /// </summary>
-    public class BinaryArrayEqualityComparer : IEqualityComparer<IBinaryObject>, IBinaryEqualityComparer,
+    public sealed class BinaryArrayEqualityComparer : IEqualityComparer<IBinaryObject>, IBinaryEqualityComparer,
         IBinaryStreamProcessor<KeyValuePair<int,int>, int>
     {
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
index aa4795e..bb26076 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
@@ -92,7 +92,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                     };
 
                 default:
-                    throw new ArgumentOutOfRangeException();
+                    throw new ArgumentOutOfRangeException("reader", type, "Invalid EqualityComparer type code");
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
index 6adb847..21c56a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
@@ -69,7 +69,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         public override bool Equals(object obj)
         {
             if (ReferenceEquals(null, obj)) return false;
-            return obj is DateTimeHolder && _item.Equals(((DateTimeHolder) obj)._item);
+            return obj is DateTimeHolder && Equals((DateTimeHolder) obj);
         }
 
         /** <inheritDoc /> */
@@ -77,5 +77,25 @@ namespace Apache.Ignite.Core.Impl.Binary
         {
             return _item.GetHashCode();
         }
+
+        /** <inheritDoc /> */
+        public static bool operator ==(DateTimeHolder left, DateTimeHolder right)
+        {
+            return left.Equals(right);
+        }
+
+        /** <inheritDoc /> */
+        public static bool operator !=(DateTimeHolder left, DateTimeHolder right)
+        {
+            return !left.Equals(right);
+        }
+
+        /// <summary>
+        /// Checks equality.
+        /// </summary>
+        private bool Equals(DateTimeHolder other)
+        {
+            return _item.Equals(other._item);
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
index 3a46515..80c1a74 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-
 namespace Apache.Ignite.Core.Impl.Binary.IO
 {
     using System;
@@ -329,19 +328,4 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         /// </summary>
         void Flush();
     }
-
-    /// <summary>
-    /// Binary stream processor.
-    /// </summary>
-    [CLSCompliant(false)]
-    public unsafe interface IBinaryStreamProcessor<in TArg, out T>
-    {
-        /// <summary>
-        /// Invokes the processor.
-        /// </summary>
-        /// <param name="data">Data.</param>
-        /// <param name="arg">Argument.</param>
-        /// <returns>Result.</returns>
-        T Invoke(byte* data, TArg arg);
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7b711a32/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStreamProcessor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStreamProcessor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStreamProcessor.cs
new file mode 100644
index 0000000..3b8bd8b
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStreamProcessor.cs
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary.IO
+{
+    using System;
+
+    /// <summary>
+    /// Binary stream processor.
+    /// </summary>
+    [CLSCompliant(false)]
+    public unsafe interface IBinaryStreamProcessor<in TArg, out T>
+    {
+        /// <summary>
+        /// Invokes the processor.
+        /// </summary>
+        /// <param name="data">Data.</param>
+        /// <param name="arg">Argument.</param>
+        /// <returns>Result.</returns>
+        T Invoke(byte* data, TArg arg);
+    }
+}
\ No newline at end of file


[40/50] [abbrv] ignite git commit: Merge ignite-3477 into ignite-db-x

Posted by sb...@apache.org.
Merge ignite-3477 into ignite-db-x


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

Branch: refs/heads/ignite-gg-11810-1
Commit: ca373d78694db25f0c6d233f4802d77fb3a7f654
Parents: 6f258b9 e7e0d87
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 17 13:43:42 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 17 13:43:42 2017 +0300

----------------------------------------------------------------------
 .../JettyRestProcessorAbstractSelfTest.java     |   2 +-
 .../internal/managers/GridManagerAdapter.java   |   3 +-
 .../processors/cache/GridCacheAdapter.java      |   7 +-
 .../cache/GridCacheConcurrentMapImpl.java       |   7 +-
 .../processors/cache/GridCacheMapEntry.java     |   8 ++
 .../GridCachePartitionExchangeManager.java      | 115 +++++++++----------
 .../cache/GridCacheSharedContext.java           |  23 +++-
 .../distributed/dht/GridDhtCacheEntry.java      |   2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   9 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   8 +-
 .../dht/GridPartitionedGetFuture.java           |   4 +-
 .../dht/GridPartitionedSingleGetFuture.java     |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  60 +++++-----
 .../dht/preloader/GridDhtForceKeysFuture.java   |  10 +-
 .../GridDhtPartitionsExchangeFuture.java        |  30 ++---
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../extras/GridCacheAttributesEntryExtras.java  |   4 +-
 .../GridCacheAttributesMvccEntryExtras.java     |   2 +-
 ...dCacheAttributesMvccObsoleteEntryExtras.java |   2 +-
 ...cheAttributesMvccObsoleteTtlEntryExtras.java |   4 +-
 .../GridCacheAttributesMvccTtlEntryExtras.java  |   4 +-
 .../GridCacheAttributesObsoleteEntryExtras.java |   2 +-
 ...idCacheAttributesObsoleteTtlEntryExtras.java |   4 +-
 .../GridCacheAttributesTtlEntryExtras.java      |   4 +-
 .../GridCacheMvccObsoleteEntryExtras.java       |   2 +-
 .../GridCacheMvccObsoleteTtlEntryExtras.java    |   4 +-
 .../extras/GridCacheMvccTtlEntryExtras.java     |   3 +-
 .../extras/GridCacheObsoleteTtlEntryExtras.java |   3 +-
 .../cache/extras/GridCacheTtlEntryExtras.java   |   2 +-
 .../util/GridCursorIteratorWrapper.java         |  24 +++-
 .../cache/CacheConfigurationLeakTest.java       |   4 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |  28 ++---
 .../GridCacheValueBytesPreloadingSelfTest.java  |  17 +--
 ...CacheLoadingConcurrentGridStartSelfTest.java |   2 +
 ...tractPartitionedByteArrayValuesSelfTest.java |   1 +
 ...GridCachePreloadRestartAbstractSelfTest.java |  12 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   6 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |  25 ++--
 .../testsuites/IgniteCacheTestSuite2.java       |  15 +--
 .../testsuites/IgniteCacheTestSuite3.java       |  13 ++-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   8 +-
 ...gniteCacheReplicatedFieldsQuerySelfTest.java |   6 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   6 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |   5 +-
 modules/yardstick/config/ignite-base-config.xml |   2 +-
 .../yardstick/cache/IgniteGetAllBenchmark.java  |  42 +++++++
 .../yardstick/cache/IgniteGetBenchmark.java     |  38 +++++-
 .../cache/jdbc/JdbcAbstractBenchmark.java       |  17 +++
 .../yardstick/cache/jdbc/JdbcPutBenchmark.java  |  17 +++
 .../cache/jdbc/JdbcPutGetBenchmark.java         |  17 +++
 .../jdbc/JdbcPutIndexedValue8Benchmark.java     |  17 +++
 .../cache/jdbc/JdbcSqlQueryBenchmark.java       |  17 +++
 .../cache/jdbc/JdbcSqlQueryJoinBenchmark.java   |  17 +++
 53 files changed, 472 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ca373d78/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ca373d78/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/ca373d78/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[26/50] [abbrv] ignite git commit: Added more metrics to Flume IgniteSink.

Posted by sb...@apache.org.
Added more metrics to Flume IgniteSink.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 6045a2469a4d2e23cad918b44e002c7de237abcb
Parents: e304b48
Author: shroman <rs...@yahoo.com>
Authored: Sat Jan 14 21:14:34 2017 +0900
Committer: shroman <rs...@yahoo.com>
Committed: Sat Jan 14 21:14:34 2017 +0900

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/stream/flume/IgniteSink.java  | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6045a246/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
----------------------------------------------------------------------
diff --git a/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java b/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
index e6e7e90..083e833 100644
--- a/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
+++ b/modules/flume/src/main/java/org/apache/ignite/stream/flume/IgniteSink.java
@@ -85,7 +85,7 @@ public class IgniteSink extends AbstractSink implements Configurable {
     }
 
     /**
-     * Starts a grid and initializes na event transformer.
+     * Starts a grid and initializes an event transformer.
      */
     @SuppressWarnings("unchecked")
     @Override synchronized public void start() {
@@ -109,9 +109,13 @@ public class IgniteSink extends AbstractSink implements Configurable {
         catch (Exception e) {
             log.error("Failed to start grid", e);
 
+            sinkCounter.incrementConnectionFailedCount();
+
             throw new FlumeException("Failed to start grid", e);
         }
 
+        sinkCounter.incrementConnectionCreatedCount();
+
         super.start();
     }
 
@@ -122,6 +126,7 @@ public class IgniteSink extends AbstractSink implements Configurable {
         if (ignite != null)
             ignite.close();
 
+        sinkCounter.incrementConnectionClosedCount();
         sinkCounter.stop();
 
         super.stop();


[48/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
index 8ec5f8f..418d28b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitExistingPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -50,10 +50,10 @@ public class SplitExistingPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.splitExistingPage(buf, mid, fwdId);
+        io.splitExistingPage(pageAddr, mid, fwdId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
index b4487fa..39f2669 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/SplitForwardPageRecord.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 
 /**
  * Split forward page record.
@@ -79,7 +80,7 @@ public class SplitForwardPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer fwdBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Split forward page record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
index 9d00d77..7cd0948 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/TrackingPageDeltaRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.TrackingPageIO;
 
 /**
@@ -70,8 +70,12 @@ public class TrackingPageDeltaRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        TrackingPageIO.VERSIONS.forPage(buf).markChanged(buf, pageIdToMark, nextSnapshotId, lastSuccessfulSnapshotId, buf.capacity());
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        TrackingPageIO.VERSIONS.forPage(pageAddr).markChanged(pageMem.pageBuffer(pageAddr),
+            pageIdToMark,
+            nextSnapshotId,
+            lastSuccessfulSnapshotId,
+            pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
index 92b72ce..c226ba2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObject.java
@@ -68,6 +68,13 @@ public interface CacheObject extends Message {
     public boolean putValue(ByteBuffer buf) throws IgniteCheckedException;
 
     /**
+     * @param addr Address tp write value to.
+     * @return Number of bytes written.
+     * @throws IgniteCheckedException If failed.
+     */
+    public int putValue(long addr) throws IgniteCheckedException;
+
+    /**
      * @param buf Buffer to write value to.
      * @param off Offset in source binary data.
      * @param len Length of the data to write.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index 1f13c6f..688b92f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@ -24,6 +24,7 @@ import java.io.ObjectOutput;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -78,6 +79,35 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    /**
+     * @param addr Write address.
+     * @param type Object type.
+     * @param valBytes Value bytes array.
+     * @param valOff Value bytes array offset.
+     * @return
+     */
+    public static int putValue(long addr, byte type, byte[] valBytes, int valOff) {
+        int off = 0;
+
+        PageUtils.putInt(addr, off, valBytes.length);
+        off += 4;
+
+        PageUtils.putByte(addr, off, type);
+        off++;
+
+        PageUtils.putBytes(addr, off, valBytes, valOff);
+        off += valBytes.length - valOff;
+
+        return off;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert valBytes != null : "Value bytes must be initialized before object is stored";
 
@@ -167,8 +197,14 @@ public abstract class CacheObjectAdapter implements CacheObject, Externalizable
      * @return {@code True} if data were successfully written.
      * @throws IgniteCheckedException If failed.
      */
-    public static boolean putValue(byte cacheObjType, final ByteBuffer buf, int off, int len,
-        byte[] valBytes, final int start) throws IgniteCheckedException {
+    public static boolean putValue(byte cacheObjType,
+        final ByteBuffer buf,
+        int off,
+        int len,
+        byte[] valBytes,
+        final int start)
+        throws IgniteCheckedException
+    {
         int dataLen = valBytes.length;
 
         if (buf.remaining() < len)

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
index b3a4117..eee6fcc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectByteArrayImpl.java
@@ -82,6 +82,11 @@ public class CacheObjectByteArrayImpl implements CacheObject, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), val, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         assert val != null : "Value is not initialized";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index 76450fb..f7e46d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -23,7 +23,6 @@ import java.util.Iterator;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
 import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
@@ -31,11 +30,14 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.pagemem.FullPageId;
+import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRowAdapter;
+import org.apache.ignite.internal.processors.cache.database.CacheSearchRow;
 import org.apache.ignite.internal.processors.cache.database.IgniteCacheDatabaseSharedManager;
 import org.apache.ignite.internal.processors.cache.database.RootPage;
 import org.apache.ignite.internal.processors.cache.database.RowStore;
@@ -44,6 +46,8 @@ import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
@@ -69,6 +73,8 @@ import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.FLAG_IDX;
 import static org.apache.ignite.internal.pagemem.PageIdAllocator.INDEX_PARTITION;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.itemId;
+import static org.apache.ignite.internal.pagemem.PageIdUtils.pageId;
 
 /**
  *
@@ -880,7 +886,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             CacheObject val,
             GridCacheVersion ver,
             long expireTime) throws IgniteCheckedException {
-            DataRow dataRow = new DataRow(key.hashCode(), key, val, ver, p, expireTime);
+            DataRow dataRow = new DataRow(key, val, ver, p, expireTime);
 
             // Make sure value bytes initialized.
             key.valueBytes(cctx.cacheObjectContext());
@@ -894,7 +900,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
                 assert dataRow.link() != 0 : dataRow;
 
-                DataRow old = dataTree.put(dataRow);
+                CacheDataRow old = dataTree.put(dataRow);
 
                 if (old == null)
                     storageSize.incrementAndGet();
@@ -933,7 +939,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
                 throw new NodeStoppingException("Operation has been cancelled (node is stopping).");
 
             try {
-                DataRow dataRow = dataTree.remove(new KeySearchRow(key.hashCode(), key, 0));
+                CacheDataRow dataRow = dataTree.remove(new SearchRow(key));
 
                 CacheObject val = null;
                 GridCacheVersion ver = null;
@@ -970,7 +976,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public CacheDataRow find(KeyCacheObject key)
             throws IgniteCheckedException {
-            return dataTree.findOne(new KeySearchRow(key.hashCode(), key, 0));
+            return dataTree.findOne(new SearchRow(key));
         }
 
         /** {@inheritDoc} */
@@ -981,14 +987,14 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         /** {@inheritDoc} */
         @Override public GridCursor<? extends CacheDataRow> cursor(KeyCacheObject lower,
             KeyCacheObject upper) throws IgniteCheckedException {
-            KeySearchRow lowerRow = null;
-            KeySearchRow upperRow = null;
+            SearchRow lowerRow = null;
+            SearchRow upperRow = null;
 
             if (lower != null)
-                lowerRow = new KeySearchRow(lower.hashCode(), lower, 0);
+                lowerRow = new SearchRow(lower);
 
             if (upper != null)
-                upperRow = new KeySearchRow(upper.hashCode(), upper, 0);
+                upperRow = new SearchRow(upper);
 
             return dataTree.find(lowerRow, upperRow);
         }
@@ -1034,82 +1040,81 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    private class KeySearchRow extends CacheDataRowAdapter {
+    private static class SearchRow implements CacheSearchRow {
         /** */
-        protected int hash;
+        private final KeyCacheObject key;
+
+        /** */
+        private final int hash;
 
         /**
-         * @param hash Hash code.
          * @param key Key.
-         * @param link Link.
          */
-        KeySearchRow(int hash, KeyCacheObject key, long link) {
-            super(link);
-
+        SearchRow(KeyCacheObject key) {
             this.key = key;
-            this.hash = hash;
-        }
 
-        /**
-         * Init data.
-         *
-         * @param keyOnly Initialize only key.
-         */
-        protected final void initData(boolean keyOnly) {
-            if (key != null)
-                return;
-
-            assert link() != 0;
-
-            try {
-                initFromLink(cctx, keyOnly);
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e.getMessage(), e);
-            }
+            hash = key.hashCode();
         }
 
-        /**
-         * @return Key.
-         */
+        /** {@inheritDoc} */
         @Override public KeyCacheObject key() {
-            initData(true);
-
             return key;
         }
+
+        /** {@inheritDoc} */
+        @Override public long link() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
+        }
     }
 
     /**
      *
      */
-    private class DataRow extends KeySearchRow {
+    private class DataRow extends CacheDataRowAdapter {
         /** */
         protected int part = -1;
 
+        /** */
+        protected int hash;
+
         /**
          * @param hash Hash code.
          * @param link Link.
+         * @param keyOnly If {@code true} initializes only key.
          */
-        DataRow(int hash, long link) {
-            super(hash, null, link);
+        DataRow(int hash, long link, boolean keyOnly) {
+            super(link);
+
+            this.hash = hash;
 
             part = PageIdUtils.partId(link);
 
-            // We can not init data row lazily because underlying buffer can be concurrently cleared.
-            initData(false);
+            try {
+                // We can not init data row lazily because underlying buffer can be concurrently cleared.
+                initFromLink(cctx, keyOnly);
+            }
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
+            }
         }
 
         /**
-         * @param hash Hash code.
          * @param key Key.
          * @param val Value.
          * @param ver Version.
          * @param part Partition.
          * @param expireTime Expire time.
          */
-        DataRow(int hash, KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
-            super(hash, key, 0);
+        DataRow(KeyCacheObject key, CacheObject val, GridCacheVersion ver, int part, long expireTime) {
+            super(0);
 
+            this.hash = key.hashCode();
+            this.key = key;
             this.val = val;
             this.ver = ver;
             this.part = part;
@@ -1122,6 +1127,11 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
+        @Override public int hash() {
+            return hash;
+        }
+
+        /** {@inheritDoc} */
         @Override public void link(long link) {
             this.link = link;
         }
@@ -1130,7 +1140,7 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     /**
      *
      */
-    protected static class CacheDataTree extends BPlusTree<KeySearchRow, DataRow> {
+    protected static class CacheDataTree extends BPlusTree<CacheSearchRow, CacheDataRow> {
         /** */
         private final CacheDataRowStore rowStore;
 
@@ -1169,38 +1179,76 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx, KeySearchRow row)
+        @Override protected int compare(BPlusIO<CacheSearchRow> io, long pageAddr, int idx, CacheSearchRow row)
             throws IgniteCheckedException {
-            int hash = ((RowLinkIO)io).getHash(buf, idx);
+            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
 
-            int cmp = Integer.compare(hash, row.hash);
+            int cmp = Integer.compare(hash, row.hash());
 
             if (cmp != 0)
                 return cmp;
 
-            KeySearchRow row0 = io.getLookupRow(this, buf, idx);
+            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
 
-            return compareKeys(row0.key(), row.key());
-        }
+            assert row.key() != null : row;
 
-        /** {@inheritDoc} */
-        @Override protected DataRow getRow(BPlusIO<KeySearchRow> io, ByteBuffer buf, int idx)
-            throws IgniteCheckedException {
-            int hash = ((RowLinkIO)io).getHash(buf, idx);
-            long link = ((RowLinkIO)io).getLink(buf, idx);
-
-            return rowStore.dataRow(hash, link);
+            return compareKeys(row.key(), link);
         }
 
         /**
-         * @param key1 First key.
-         * @param key2 Second key.
+         * @param key Key.
+         * @param link Link.
          * @return Compare result.
          * @throws IgniteCheckedException If failed.
          */
-        private int compareKeys(CacheObject key1, CacheObject key2) throws IgniteCheckedException {
-            byte[] bytes1 = key1.valueBytes(cctx.cacheObjectContext());
-            byte[] bytes2 = key2.valueBytes(cctx.cacheObjectContext());
+        private int compareKeys(KeyCacheObject key, final long link) throws IgniteCheckedException {
+            byte[] bytes = key.valueBytes(cctx.cacheObjectContext());
+
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
+            try (Page page = page(pageId(link))) {
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
+
+                assert pageAddr != 0L : link;
+
+                try {
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
+
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(link),
+                        pageMem.pageSize());
+
+                    if (data.nextLink() == 0) {
+                        long addr = pageAddr + data.offset();
+
+                        int len = PageUtils.getInt(addr, 0);
+
+                        int size = Math.min(bytes.length, len);
+
+                        addr += 5; // Skip length and type byte.
+
+                        for (int i = 0; i < size; i++) {
+                            byte b1 = PageUtils.getByte(addr, i);
+                            byte b2 = bytes[i];
+
+                            if (b1 != b2)
+                                return b1 > b2 ? 1 : -1;
+                        }
+
+                        return Integer.compare(len, bytes.length);
+                    }
+                }
+                finally {
+                    page.releaseRead();
+                }
+            }
+
+            // TODO GG-11768.
+            CacheDataRowAdapter other = new CacheDataRowAdapter(link);
+            other.initFromLink(cctx, true);
+
+            byte[] bytes1 = other.key().valueBytes(cctx.cacheObjectContext());
+            byte[] bytes2 = key.valueBytes(cctx.cacheObjectContext());
 
             int len = Math.min(bytes1.length, bytes2.length);
 
@@ -1214,6 +1262,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
 
             return Integer.compare(bytes1.length, bytes2.length);
         }
+
+        /** {@inheritDoc} */
+        @Override protected CacheDataRow getRow(BPlusIO<CacheSearchRow> io, long pageAddr, int idx)
+            throws IgniteCheckedException {
+            int hash = ((RowLinkIO)io).getHash(pageAddr, idx);
+            long link = ((RowLinkIO)io).getLink(pageAddr, idx);
+
+            return rowStore.dataRow(hash, link);
+        }
     }
 
     /**
@@ -1233,8 +1290,8 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Search row.
          */
-        private KeySearchRow keySearchRow(int hash, long link) {
-            return new KeySearchRow(hash, null, link);
+        private CacheSearchRow keySearchRow(int hash, long link) {
+            return new DataRow(hash, link, true);
         }
 
         /**
@@ -1242,8 +1299,8 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
          * @param link Link.
          * @return Data row.
          */
-        private DataRow dataRow(int hash, long link) {
-            return new DataRow(hash, link);
+        private CacheDataRow dataRow(int hash, long link) {
+            return new DataRow(hash, link, false);
         }
     }
 
@@ -1259,28 +1316,39 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
     }
 
     /**
+     * @param pageAddr Page address.
+     * @param off Offset.
+     * @param link Link.
+     * @param hash Hash.
+     */
+    private static void store0(long pageAddr, int off, long link, int hash) {
+        PageUtils.putLong(pageAddr, off, link);
+        PageUtils.putInt(pageAddr, off + 8, hash);
+    }
+
+    /**
      *
      */
     private interface RowLinkIO {
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Row link.
          */
-        public long getLink(ByteBuffer buf, int idx);
+        public long getLink(long pageAddr, int idx);
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Key hash code.
          */
-        public int getHash(ByteBuffer buf, int idx);
+        public int getHash(long pageAddr, int idx);
     }
 
     /**
      *
      */
-    public static final class DataInnerIO extends BPlusInnerIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataInnerIO extends BPlusInnerIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataInnerIO> VERSIONS = new IOVersions<>(
             new DataInnerIO(1)
@@ -1294,46 +1362,53 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, CacheSearchRow row) throws IgniteCheckedException {
             assert row.link() != 0;
 
-            store0(buf, off, row.link(), row.hash);
+            store0(buf, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
-            int hash = getHash(buf, idx);
-            long link = getLink(buf, idx);
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
+            assert row.link() != 0;
+
+            store0(pageAddr, off, row.link(), row.hash());
+        }
+
+        /** {@inheritDoc} */
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long pageAddr, int idx) {
+            int hash = getHash(pageAddr, idx);
+            long link = getLink(pageAddr, idx);
 
             return ((CacheDataTree)tree).rowStore.keySearchRow(hash, link);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
-            int hash = ((RowLinkIO)srcIo).getHash(src, srcIdx);
-            long link = ((RowLinkIO)srcIo).getLink(src, srcIdx);
+            int hash = ((RowLinkIO)srcIo).getHash(srcPageAddr, srcIdx);
+            long link = ((RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx);
 
-            store0(dst, offset(dstIdx), link, hash);
+            store0(dstPageAddr, offset(dstIdx), link, hash);
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            assert idx < getCount(buf) : idx;
+        @Override public long getLink(long pageAddr, int idx) {
+            assert idx < getCount(pageAddr) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long pageAddr, int idx) {
+            return PageUtils.getInt(pageAddr, offset(idx) + 8);
         }
     }
 
     /**
      *
      */
-    public static final class DataLeafIO extends BPlusLeafIO<KeySearchRow> implements RowLinkIO {
+    public static final class DataLeafIO extends BPlusLeafIO<CacheSearchRow> implements RowLinkIO {
         /** */
         public static final IOVersions<DataLeafIO> VERSIONS = new IOVersions<>(
             new DataLeafIO(1)
@@ -1347,20 +1422,27 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, KeySearchRow row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, CacheSearchRow row) throws IgniteCheckedException {
+            assert row.link() != 0;
+
+            store0(buf, off, row.link(), row.hash());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, CacheSearchRow row) {
             assert row.link() != 0;
 
-            store0(buf, off, row.link(), row.hash);
+            store0(pageAddr, off, row.link(), row.hash());
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<KeySearchRow> srcIo, ByteBuffer src,
+        @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<CacheSearchRow> srcIo, long srcPageAddr,
             int srcIdx) {
-            store0(dst, offset(dstIdx), getLink(src, srcIdx), getHash(src, srcIdx));
+            store0(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx), getHash(srcPageAddr, srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public KeySearchRow getLookupRow(BPlusTree<KeySearchRow, ?> tree, ByteBuffer buf, int idx) {
+        @Override public CacheSearchRow getLookupRow(BPlusTree<CacheSearchRow, ?> tree, long buf, int idx) {
             int hash = getHash(buf, idx);
             long link = getLink(buf, idx);
 
@@ -1368,15 +1450,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            assert idx < getCount(buf) : idx;
+        @Override public long getLink(long pageAddr, int idx) {
+            assert idx < getCount(pageAddr) : idx;
 
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public int getHash(ByteBuffer buf, int idx) {
-            return buf.getInt(offset(idx) + 8);
+        @Override public int getHash(long pageAddr, int idx) {
+            return PageUtils.getInt(pageAddr, offset(idx) + 8);
         }
     }
 
@@ -1471,9 +1553,9 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<PendingRow> io, ByteBuffer buf, int idx, PendingRow row)
+        @Override protected int compare(BPlusIO<PendingRow> io, long pageAddr, int idx, PendingRow row)
             throws IgniteCheckedException {
-            long expireTime = ((PendingRowIO)io).getExpireTime(buf, idx);
+            long expireTime = ((PendingRowIO)io).getExpireTime(pageAddr, idx);
 
             int cmp = Long.compare(expireTime, row.expireTime);
 
@@ -1483,15 +1565,15 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
             if (row.link == 0L)
                 return 0;
 
-            long link = ((PendingRowIO)io).getLink(buf, idx);
+            long link = ((PendingRowIO)io).getLink(pageAddr, idx);
 
             return Long.compare(link, row.link);
         }
 
         /** {@inheritDoc} */
-        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, ByteBuffer buf, int idx)
+        @Override protected PendingRow getRow(BPlusIO<PendingRow> io, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return io.getLookupRow(this, buf, idx);
+            return io.getLookupRow(this, pageAddr, idx);
         }
     }
 
@@ -1500,18 +1582,18 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
      */
     private interface PendingRowIO {
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Expire time.
          */
-        long getExpireTime(ByteBuffer buf, int idx);
+        long getExpireTime(long pageAddr, int idx);
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @return Link.
          */
-        long getLink(ByteBuffer buf, int idx);
+        long getLink(long pageAddr, int idx);
     }
 
     /**
@@ -1540,34 +1622,45 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void storeByOffset(long pageAddr, int off, PendingRow row) throws IgniteCheckedException {
+            assert row.link != 0;
+            assert row.expireTime != 0;
+
+            PageUtils.putLong(pageAddr, off, row.expireTime);
+            PageUtils.putLong(pageAddr, off + 8, row.link);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long srcPageAddr,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
-            long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
-            long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
+            long link = ((PendingRowIO)srcIo).getLink(srcPageAddr, srcIdx);
+            long expireTime = ((PendingRowIO)srcIo).getExpireTime(srcPageAddr, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dstPageAddr, dstOff, expireTime);
+            PageUtils.putLong(dstPageAddr, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
+            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx,
+                getExpireTime(pageAddr, idx),
+                getLink(pageAddr, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx) + 8);
         }
     }
 
@@ -1597,34 +1690,45 @@ public class IgniteCacheOffheapManagerImpl extends GridCacheManagerAdapter imple
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst,
+        @Override public void storeByOffset(long pageAddr, int off, PendingRow row) throws IgniteCheckedException {
+            assert row.link != 0;
+            assert row.expireTime != 0;
+
+            PageUtils.putLong(pageAddr, off, row.expireTime);
+            PageUtils.putLong(pageAddr, off + 8, row.link);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(long dstPageAddr,
             int dstIdx,
             BPlusIO<PendingRow> srcIo,
-            ByteBuffer src,
+            long srcPageAddr,
             int srcIdx) throws IgniteCheckedException {
             int dstOff = offset(dstIdx);
 
-            long link = ((PendingRowIO)srcIo).getLink(src, srcIdx);
-            long expireTime = ((PendingRowIO)srcIo).getExpireTime(src, srcIdx);
+            long link = ((PendingRowIO)srcIo).getLink(srcPageAddr, srcIdx);
+            long expireTime = ((PendingRowIO)srcIo).getExpireTime(srcPageAddr, srcIdx);
 
-            dst.putLong(dstOff, expireTime);
-            dst.putLong(dstOff + 8, link);
+            PageUtils.putLong(dstPageAddr, dstOff, expireTime);
+            PageUtils.putLong(dstPageAddr, dstOff + 8, link);
         }
 
         /** {@inheritDoc} */
-        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, ByteBuffer buf, int idx)
+        @Override public PendingRow getLookupRow(BPlusTree<PendingRow, ?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx, getExpireTime(buf, idx), getLink(buf, idx));
+            return PendingRow.createRowWithKey(((PendingEntriesTree)tree).cctx,
+                getExpireTime(pageAddr, idx),
+                getLink(pageAddr, idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getExpireTime(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx));
+        @Override public long getExpireTime(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
-        @Override public long getLink(ByteBuffer buf, int idx) {
-            return buf.getLong(offset(idx) + 8);
+        @Override public long getLink(long pageAddr, int idx) {
+            return PageUtils.getLong(pageAddr, offset(idx) + 8);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
index d4d7020..75ab8e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRow.java
@@ -18,18 +18,12 @@
 package org.apache.ignite.internal.processors.cache.database;
 
 import org.apache.ignite.internal.processors.cache.CacheObject;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 
 /**
  * Cache data row.
  */
-public interface CacheDataRow {
-    /**
-     * @return Cache key.
-     */
-    public KeyCacheObject key();
-
+public interface CacheDataRow extends CacheSearchRow {
     /**
      * @return Cache value.
      */
@@ -51,11 +45,6 @@ public interface CacheDataRow {
     public int partition();
 
     /**
-     * @return Link for this row.
-     */
-    public long link();
-
-    /**
      * @param link Link for this row.
      */
     public void link(long link);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
index b5babc4..5288aad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheDataRowAdapter.java
@@ -21,14 +21,17 @@ import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IncompleteCacheObject;
 import org.apache.ignite.internal.processors.cache.IncompleteObject;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -88,20 +91,26 @@ public class CacheDataRowAdapter implements CacheDataRow {
         boolean first = true;
 
         do {
+            PageMemory pageMem = cctx.shared().database().pageMemory();
+
             try (Page page = page(pageId(nextLink), cctx)) {
-                ByteBuffer buf = page.getForRead(); // Non-empty data page must not be recycled.
+                long pageAddr = page.getForReadPointer(); // Non-empty data page must not be recycled.
 
-                assert buf != null: nextLink;
+                assert pageAddr != 0L : nextLink;
 
                 try {
-                    DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+                    DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-                    nextLink = io.setPositionAndLimitOnPayload(buf, itemId(nextLink));
+                    DataPagePayload data = io.readPayload(pageAddr,
+                        itemId(nextLink),
+                        pageMem.pageSize());
+
+                    nextLink = data.nextLink();
 
                     if (first) {
                         if (nextLink == 0) {
                             // Fast path for a single page row.
-                            readFullRow(coctx, buf, keyOnly);
+                            readFullRow(coctx, pageAddr + data.offset(), keyOnly);
 
                             return;
                         }
@@ -109,6 +118,11 @@ public class CacheDataRowAdapter implements CacheDataRow {
                         first = false;
                     }
 
+                    ByteBuffer buf = pageMem.pageBuffer(pageAddr);
+
+                    buf.position(data.offset());
+                    buf.limit(data.offset() + data.payloadSize());
+
                     incomplete = readFragment(coctx, buf, keyOnly, incomplete);
 
                     if (keyOnly && key != null)
@@ -121,7 +135,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
         }
         while(nextLink != 0);
 
-        assert isReady(): "ready";
+        assert isReady() : "ready";
     }
 
     /**
@@ -130,6 +144,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param keyOnly {@code true} If need to read only key object.
      * @param incomplete Incomplete object.
      * @throws IgniteCheckedException If failed.
+     * @return Read object.
      */
     private IncompleteObject<?> readFragment(
         CacheObjectContext coctx,
@@ -175,12 +190,23 @@ public class CacheDataRowAdapter implements CacheDataRow {
 
     /**
      * @param coctx Cache object context.
-     * @param buf Buffer.
+     * @param addr Address.
      * @param keyOnly {@code true} If need to read only key object.
      * @throws IgniteCheckedException If failed.
      */
-    private void readFullRow(CacheObjectContext coctx, ByteBuffer buf, boolean keyOnly) throws IgniteCheckedException {
-        key = coctx.processor().toKeyCacheObject(coctx, buf);
+    private void readFullRow(CacheObjectContext coctx, long addr, boolean keyOnly) throws IgniteCheckedException {
+        int off = 0;
+
+        int len = PageUtils.getInt(addr, off);
+        off += 4;
+
+        byte type = PageUtils.getByte(addr, off);
+        off++;
+
+        byte[] bytes = PageUtils.getBytes(addr, off, len);
+        off += len;
+
+        key = coctx.processor().toKeyCacheObject(coctx, type, bytes);
 
         if (keyOnly) {
             assert key != null: "key";
@@ -188,9 +214,22 @@ public class CacheDataRowAdapter implements CacheDataRow {
             return;
         }
 
-        val = coctx.processor().toCacheObject(coctx, buf);
-        ver = CacheVersionIO.read(buf, false);
-        expireTime = buf.getLong();
+        len = PageUtils.getInt(addr, off);
+        off += 4;
+
+        type = PageUtils.getByte(addr, off);
+        off++;
+
+        bytes = PageUtils.getBytes(addr, off, len);
+        off += len;
+
+        val = coctx.processor().toCacheObject(coctx, type, bytes);
+
+        ver = CacheVersionIO.read(addr + off, false);
+
+        off += CacheVersionIO.size(ver, false);
+
+        expireTime = PageUtils.getLong(addr, off);
 
         assert isReady(): "ready";
     }
@@ -249,6 +288,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteExpireTime(
         ByteBuffer buf,
@@ -292,6 +332,7 @@ public class CacheDataRowAdapter implements CacheDataRow {
      * @param buf Buffer.
      * @param incomplete Incomplete object.
      * @return Incomplete object.
+     * @throws IgniteCheckedException If failed.
      */
     private IncompleteObject<?> readIncompleteVersion(
         ByteBuffer buf,
@@ -385,6 +426,11 @@ public class CacheDataRowAdapter implements CacheDataRow {
     }
 
     /** {@inheritDoc} */
+    @Override public int hash() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheDataRowAdapter.class, this, "link", U.hexLong(link));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.java
new file mode 100644
index 0000000..d51cf0e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/CacheSearchRow.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.internal.processors.cache.database;
+
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+
+/**
+ *
+ */
+public interface CacheSearchRow {
+    /**
+     * @return Cache key.
+     */
+    public KeyCacheObject key();
+
+    /**
+     * @return Link for this row.
+     */
+    public long link();
+
+    /**
+     * @return Key hash code.
+     */
+    public int hash();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
index 5fd64b0..f47a697 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/DataStructure.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database;
 
-import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.ignite.IgniteCheckedException;
@@ -131,35 +130,35 @@ public abstract class DataStructure implements PageLockListener {
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer tryWriteLock(Page page) {
+    protected final long tryWriteLock(Page page) {
         return PageHandler.writeLock(page, this, true);
     }
 
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer writeLock(Page page) {
+    protected final long writeLock(Page page) {
         return PageHandler.writeLock(page, this, false);
     }
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param dirty Dirty page.
      */
-    protected final void writeUnlock(Page page, ByteBuffer buf, boolean dirty) {
-        PageHandler.writeUnlock(page, buf, this, dirty);
+    protected final void writeUnlock(Page page, long pageAddr, boolean dirty) {
+        PageHandler.writeUnlock(page, pageAddr, this, dirty);
     }
 
     /**
      * @param page Page.
-     * @return Buffer.
+     * @return Page address.
      */
-    protected final ByteBuffer readLock(Page page) {
+    protected final long readLock(Page page) {
         return PageHandler.readLock(page, this);
     }
 
@@ -167,22 +166,29 @@ public abstract class DataStructure implements PageLockListener {
      * @param page Page.
      * @param buf Buffer.
      */
-    protected final void readUnlock(Page page, ByteBuffer buf) {
+    protected final void readUnlock(Page page, long buf) {
         PageHandler.readUnlock(page, buf, this);
     }
 
+    /**
+     * @return Page size.
+     */
+    protected final int pageSize() {
+        return pageMem.pageSize();
+    }
+
     /** {@inheritDoc} */
     @Override public void onBeforeWriteLock(Page page) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteLock(Page page, ByteBuffer buf) {
+    @Override public void onWriteLock(Page page, long pageAddr) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
+    @Override public void onWriteUnlock(Page page, long pageAddr) {
         // No-op.
     }
 
@@ -192,12 +198,12 @@ public abstract class DataStructure implements PageLockListener {
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadLock(Page page, ByteBuffer buf) {
+    @Override public void onReadLock(Page page, long pageAddr) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onReadUnlock(Page page, ByteBuffer buf) {
+    @Override public void onReadUnlock(Page page, long pageAddr) {
         // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
index 18b3a1f..9c10057 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/IgniteCacheDatabaseSharedManager.java
@@ -256,7 +256,7 @@ public class IgniteCacheDatabaseSharedManager extends GridCacheSharedManagerAdap
                 true,
                 sizes);
 
-        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize());
+        return new PageMemoryNoStoreImpl(log, memProvider, cctx, dbCfg.getPageSize(), false);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
index 26151ac..cf6decb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/MetadataStorage.java
@@ -24,6 +24,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
@@ -31,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
+import org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
@@ -190,19 +192,19 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(final BPlusIO<IndexItem> io, final ByteBuffer buf, final int idx,
+        @Override protected int compare(final BPlusIO<IndexItem> io, final long pageAddr, final int idx,
             final IndexItem row) throws IgniteCheckedException {
             final int off = ((IndexIO)io).getOffset(idx);
 
             int shift = 0;
 
             // Compare index names.
-            final byte len = buf.get(off + shift);
+            final byte len = PageUtils.getByte(pageAddr, off + shift);
 
             shift += BYTE_LEN;
 
             for (int i = 0; i < len && i < row.idxName.length; i++) {
-                final int cmp = Byte.compare(buf.get(off + i + shift), row.idxName[i]);
+                final int cmp = Byte.compare(PageUtils.getByte(pageAddr, off + i + shift), row.idxName[i]);
 
                 if (cmp != 0)
                     return cmp;
@@ -212,9 +214,9 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final ByteBuffer buf,
+        @Override protected IndexItem getRow(final BPlusIO<IndexItem> io, final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, ((IndexIO)io).getOffset(idx));
+            return readRow(pageAddr, ((IndexIO)io).getOffset(idx));
         }
     }
 
@@ -275,78 +277,78 @@ public class MetadataStorage implements MetaStore {
     }
 
     /**
-     * Copy row data.
+     * Store row to buffer.
      *
-     * @param dst Destination buffer.
-     * @param dstOff Destination buf offset.
-     * @param src Source buffer.
-     * @param srcOff Src buf offset.
+     * @param pageAddr Page address.
+     * @param off Offset in buf.
+     * @param row Row to store.
      */
     private static void storeRow(
-        final ByteBuffer dst,
-        final int dstOff,
-        final ByteBuffer src,
-        final int srcOff
+        final long pageAddr,
+        int off,
+        final IndexItem row
     ) {
-        int srcOrigPos = src.position();
-        int dstOrigPos = dst.position();
-
-        try {
-            src.position(srcOff);
-            dst.position(dstOff);
-
-            // Index name length.
-            final byte len = src.get();
+        // Index name length.
+        PageUtils.putByte(pageAddr, off, (byte)row.idxName.length);
+        off++;
 
-            dst.put(len);
+        // Index name.
+        PageUtils.putBytes(pageAddr, off, row.idxName);
+        off += row.idxName.length;
 
-            int lim = src.limit();
+        // Page ID.
+        PageUtils.putLong(pageAddr, off, row.pageId);
+    }
 
-            src.limit(src.position() + len);
+    /**
+     * Copy row data.
+     *
+     * @param dstPageAddr Destination page address.
+     * @param dstOff Destination buf offset.
+     * @param srcPageAddr Source page address.
+     * @param srcOff Src buf offset.
+     */
+    private static void storeRow(
+        final long dstPageAddr,
+        int dstOff,
+        final long srcPageAddr,
+        int srcOff
+    ) {
+        // Index name length.
+        final byte len = PageUtils.getByte(srcPageAddr, srcOff);
+        srcOff++;
 
-            // Index name.
-            dst.put(src);
+        PageUtils.putByte(dstPageAddr, dstOff, len);
+        dstOff++;
 
-            src.limit(lim);
+        PageHandler.copyMemory(srcPageAddr, srcOff, dstPageAddr, dstOff, len);
+        srcOff += len;
+        dstOff += len;
 
-            // Page ID.
-            dst.putLong(src.getLong());
-        }
-        finally {
-            src.position(srcOrigPos);
-            dst.position(dstOrigPos);
-        }
+        // Page ID.
+        PageUtils.putLong(dstPageAddr, dstOff, PageUtils.getLong(srcPageAddr, srcOff));
     }
 
     /**
      * Read row from buffer.
      *
-     * @param buf Buffer to read.
-     * @param off Offset in buf.
+     * @param pageAddr Page address.
+     * @param off Offset.
      * @return Read row.
      */
-    private static IndexItem readRow(final ByteBuffer buf, final int off) {
-        int origOff = buf.position();
-
-        try {
-            buf.position(off);
-
-            // Index name length.
-            final int len = buf.get() & 0xFF;
-
-            // Index name.
-            final byte[] idxName = new byte[len];
+    private static IndexItem readRow(final long pageAddr, int off) {
+        // Index name length.
+        final int len = PageUtils.getByte(pageAddr, off) & 0xFF;
+        off++;
 
-            buf.get(idxName);
+        // Index name.
+        final byte[] idxName = PageUtils.getBytes(pageAddr, off, len);
+        off += len;
 
-            // Page ID.
-            final long pageId = buf.getLong();
+        // Page ID.
+        final long pageId = PageUtils.getLong(pageAddr, off);
 
-            return new IndexItem(idxName, pageId);
-        }
-        finally {
-            buf.position(origOff);
-        }
+        return new IndexItem(idxName, pageId);
     }
 
     /**
@@ -383,16 +385,21 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void storeByOffset(long pageAddr, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(pageAddr, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr, final int dstIdx, final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
             final int srcIdx) throws IgniteCheckedException {
-            storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, offset(idx));
+            return readRow(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */
@@ -424,16 +431,24 @@ public class MetadataStorage implements MetaStore {
         }
 
         /** {@inheritDoc} */
-        @Override public void store(final ByteBuffer dst, final int dstIdx, final BPlusIO<IndexItem> srcIo,
-            final ByteBuffer src,
+        @Override public void storeByOffset(long buf, int off, IndexItem row) throws IgniteCheckedException {
+            storeRow(buf, off, row);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void store(final long dstPageAddr,
+            final int dstIdx,
+            final BPlusIO<IndexItem> srcIo,
+            final long srcPageAddr,
             final int srcIdx) throws IgniteCheckedException {
-            storeRow(dst, offset(dstIdx), src, ((IndexIO)srcIo).getOffset(srcIdx));
+            storeRow(dstPageAddr, offset(dstIdx), srcPageAddr, ((IndexIO)srcIo).getOffset(srcIdx));
         }
 
         /** {@inheritDoc} */
-        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree, final ByteBuffer buf,
+        @Override public IndexItem getLookupRow(final BPlusTree<IndexItem, ?> tree,
+            final long pageAddr,
             final int idx) throws IgniteCheckedException {
-            return readRow(buf, offset(idx));
+            return readRow(pageAddr, offset(idx));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
index 6a29027..6c1b21b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/FreeListImpl.java
@@ -17,13 +17,13 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertFragmentRecord;
 import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageInsertRecord;
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.pagemem.wal.record.delta.DataPageRemoveRecord;
 import org.apache.ignite.internal.processors.cache.database.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.database.tree.io.CacheVersionIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
+import org.apache.ignite.internal.processors.cache.database.tree.io.DataPagePayload;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseBag;
 import org.apache.ignite.internal.processors.cache.database.tree.reuse.ReuseList;
@@ -72,26 +73,26 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     /** */
     private final PageHandler<CacheDataRow, Integer> writeRow =
         new PageHandler<CacheDataRow, Integer>() {
-            @Override public Integer run(Page page, PageIO iox, ByteBuffer buf, CacheDataRow row, int written)
+            @Override public Integer run(Page page, PageIO iox, long pageAddr, CacheDataRow row, int written)
                 throws IgniteCheckedException {
                 DataPageIO io = (DataPageIO)iox;
 
                 int rowSize = getRowSize(row);
-                int oldFreeSpace = io.getFreeSpace(buf);
+                int oldFreeSpace = io.getFreeSpace(pageAddr);
 
                 assert oldFreeSpace > 0 : oldFreeSpace;
 
                 // If the full row does not fit into this page write only a fragment.
-                written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, buf, io, row, rowSize):
-                    addRowFragment(page, buf, io, row, written, rowSize);
+                written = (written == 0 && oldFreeSpace >= rowSize) ? addRow(page, pageAddr, io, row, rowSize):
+                    addRowFragment(page, pageAddr, io, row, written, rowSize);
 
                 // Reread free space after update.
-                int newFreeSpace = io.getFreeSpace(buf);
+                int newFreeSpace = io.getFreeSpace(pageAddr);
 
                 if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                     int bucket = bucket(newFreeSpace, false);
 
-                    put(null, page, buf, bucket);
+                    put(null, page, pageAddr, bucket);
                 }
 
                 // Avoid boxing with garbage generation for usual case.
@@ -109,24 +110,22 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRow(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int rowSize
             ) throws IgniteCheckedException {
-                // TODO: context parameter.
-                io.addRow(buf, row, rowSize);
+                io.addRow(buf, row, rowSize, pageSize());
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[rowSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
 
-                    assert buf.remaining() == rowSize;
+                    assert data.payloadSize() == rowSize;
 
-                    buf.get(payload);
-                    buf.position(0);
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, rowSize);
 
                     wal.log(new DataPageInsertRecord(
                         cacheId,
@@ -149,7 +148,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
              */
             private int addRowFragment(
                 Page page,
-                ByteBuffer buf,
+                long buf,
                 DataPageIO io,
                 CacheDataRow row,
                 int written,
@@ -158,17 +157,17 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // Read last link before the fragment write, because it will be updated there.
                 long lastLink = row.link();
 
-                int payloadSize = io.addRowFragment(buf, row, written, rowSize);
+                int payloadSize = io.addRowFragment(pageMem, buf, row, written, rowSize, pageSize());
 
-                assert payloadSize > 0: payloadSize;
+                assert payloadSize > 0 : payloadSize;
 
                 if (isWalDeltaRecordNeeded(wal, page)) {
                     // TODO This record must contain only a reference to a logical WAL record with the actual data.
                     byte[] payload = new byte[payloadSize];
 
-                    io.setPositionAndLimitOnPayload(buf, PageIdUtils.itemId(row.link()));
-                    buf.get(payload);
-                    buf.position(0);
+                    DataPagePayload data = io.readPayload(buf, PageIdUtils.itemId(row.link()), pageSize());
+
+                    PageUtils.getBytes(buf, data.offset(), payload, 0, payloadSize);
 
                     wal.log(new DataPageInsertFragmentRecord(cacheId, page.id(), payload, lastLink));
                 }
@@ -179,15 +178,15 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
     /** */
     private final PageHandler<Void, Long> rmvRow = new PageHandler<Void, Long>() {
-        @Override public Long run(Page page, PageIO iox, ByteBuffer buf, Void arg, int itemId)
+        @Override public Long run(Page page, PageIO iox, long pageAddr, Void arg, int itemId)
             throws IgniteCheckedException {
             DataPageIO io = (DataPageIO)iox;
 
-            int oldFreeSpace = io.getFreeSpace(buf);
+            int oldFreeSpace = io.getFreeSpace(pageAddr);
 
             assert oldFreeSpace >= 0: oldFreeSpace;
 
-            long nextLink = io.removeRow(buf, itemId);
+            long nextLink = io.removeRow(pageAddr, itemId, pageSize());
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new DataPageRemoveRecord(cacheId, page.id(), itemId));
@@ -200,7 +199,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 //                    put(null, page, buf, REUSE_BUCKET);
 //            }
 
-            int newFreeSpace = io.getFreeSpace(buf);
+            int newFreeSpace = io.getFreeSpace(pageAddr);
 
             if (newFreeSpace > MIN_PAGE_FREE_SPACE) {
                 int newBucket = bucket(newFreeSpace, false);
@@ -210,12 +209,12 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
 
                     if (oldBucket != newBucket) {
                         // It is possible that page was concurrently taken for put, in this case put will handle bucket change.
-                        if (removeDataPage(page, buf, io, oldBucket))
-                            put(null, page, buf, newBucket);
+                        if (removeDataPage(page, pageAddr, io, oldBucket))
+                            put(null, page, pageAddr, newBucket);
                     }
                 }
                 else
-                    put(null, page, buf, newBucket);
+                    put(null, page, pageAddr, newBucket);
             }
 
             // For common case boxed 0L will be cached inside of Long, so no garbage will be produced.
@@ -326,7 +325,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
                 // If it is an existing page, we do not need to initialize it.
                 DataPageIO init = reuseBucket || pageId == 0L ? DataPageIO.VERSIONS.latest() : null;
 
-                written = writePage(page, this, writeRow, init, wal, row, written, FAIL_I);
+                written = writePage(pageMem, page, this, writeRow, init, wal, row, written, FAIL_I);
 
                 assert written != FAIL_I; // We can't fail here.
             }
@@ -344,7 +343,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
         long nextLink;
 
         try (Page page = pageMem.page(cacheId, pageId)) {
-            nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+            nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
             assert nextLink != FAIL_L; // Can't fail here.
         }
@@ -354,7 +353,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
             pageId = PageIdUtils.pageId(nextLink);
 
             try (Page page = pageMem.page(cacheId, pageId)) {
-                nextLink = writePage(page, this, rmvRow, null, itemId, FAIL_L);
+                nextLink = writePage(pageMem, page, this, rmvRow, null, itemId, FAIL_L);
 
                 assert nextLink != FAIL_L; // Can't fail here.
             }
@@ -380,7 +379,7 @@ public class FreeListImpl extends PagesList implements FreeList, ReuseList {
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
         assert reuseList == this: "not allowed to be a reuse list";
 
-        put(bag, null, null, REUSE_BUCKET);
+        put(bag, null, 0L, REUSE_BUCKET);
     }
 
     /** {@inheritDoc} */


[47/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
index 4da44bc..e44838d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/PagesList.java
@@ -86,17 +86,17 @@ public abstract class PagesList extends DataStructure {
 
     /** */
     private final PageHandler<Void, Boolean> cutTail = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO pageIo, ByteBuffer buf, Void ignore, int bucket)
+        @Override public Boolean run(Page page, PageIO pageIo, long pageAddr, Void ignore, int bucket)
             throws IgniteCheckedException {
-            assert getPageId(buf) == page.id();
+            assert getPageId(pageAddr) == page.id();
 
             PagesListNodeIO io = (PagesListNodeIO)pageIo;
 
-            long tailId = io.getNextId(buf);
+            long tailId = io.getNextId(pageAddr);
 
             assert tailId != 0;
 
-            io.setNextId(buf, 0L);
+            io.setNextId(pageAddr, 0L);
 
             if (isWalDeltaRecordNeeded(wal, page))
                 wal.log(new PagesListSetNextRecord(cacheId, page.id(), 0L));
@@ -140,7 +140,7 @@ public abstract class PagesList extends DataStructure {
         if (metaPageId != 0L) {
             if (initNew) {
                 try (Page page = page(metaPageId)) {
-                    initPage(page, this, PagesListMetaIO.VERSIONS.latest(), wal);
+                    initPage(pageMem, page, this, PagesListMetaIO.VERSIONS.latest(), wal);
                 }
             }
             else {
@@ -150,16 +150,16 @@ public abstract class PagesList extends DataStructure {
 
                 while (nextPageId != 0) {
                     try (Page page = page(nextPageId)) {
-                        ByteBuffer buf = readLock(page); // No concurrent recycling on init.
+                        long pageAddr = readLock(page); // No concurrent recycling on init.
 
-                        assert buf != null;
+                        assert pageAddr != 0L;
 
                         try {
-                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                            PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                            io.getBucketsData(buf, bucketsData);
+                            io.getBucketsData(pageAddr, bucketsData);
 
-                            long next0 = io.getNextMetaPageId(buf);
+                            long next0 = io.getNextMetaPageId(pageAddr);
 
                             assert next0 != nextPageId :
                                 "Loop detected [next=" + U.hexLong(next0) + ", cur=" + U.hexLong(nextPageId) + ']';
@@ -167,7 +167,7 @@ public abstract class PagesList extends DataStructure {
                             nextPageId = next0;
                         }
                         finally {
-                            readUnlock(page, buf);
+                            readUnlock(page, pageAddr);
                         }
                     }
                 }
@@ -199,7 +199,7 @@ public abstract class PagesList extends DataStructure {
         assert metaPageId != 0;
 
         Page curPage = null;
-        ByteBuffer curBuf = null;
+        long curPageAddr = 0L;
         PagesListMetaIO curIo = null;
 
         long nextPageId = metaPageId;
@@ -212,39 +212,39 @@ public abstract class PagesList extends DataStructure {
                     int tailIdx = 0;
 
                     while (tailIdx < tails.length) {
-                        int written = curPage != null ? curIo.addTails(curBuf, bucket, tails, tailIdx) : 0;
+                        int written = curPage != null ? curIo.addTails(pageMem.pageSize(), curPageAddr, bucket, tails, tailIdx) : 0;
 
                         if (written == 0) {
                             if (nextPageId == 0L) {
                                 nextPageId = allocatePageNoReuse();
 
                                 if (curPage != null) {
-                                    curIo.setNextMetaPageId(curBuf, nextPageId);
+                                    curIo.setNextMetaPageId(curPageAddr, nextPageId);
 
-                                    releaseAndClose(curPage, curBuf);
+                                    releaseAndClose(curPage, curPageAddr);
                                     curPage = null;
                                 }
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
                                 curIo = PagesListMetaIO.VERSIONS.latest();
 
-                                curIo.initNewPage(curBuf, nextPageId);
+                                curIo.initNewPage(curPageAddr, nextPageId, pageSize());
                             }
                             else {
-                                releaseAndClose(curPage, curBuf);
+                                releaseAndClose(curPage, curPageAddr);
                                 curPage = null;
 
                                 curPage = page(nextPageId);
-                                curBuf = writeLock(curPage);
+                                curPageAddr = writeLock(curPage);
 
-                                curIo = PagesListMetaIO.VERSIONS.forPage(curBuf);
+                                curIo = PagesListMetaIO.VERSIONS.forPage(curPageAddr);
 
-                                curIo.resetCount(curBuf);
+                                curIo.resetCount(curPageAddr);
                             }
 
-                            nextPageId = curIo.getNextMetaPageId(curBuf);
+                            nextPageId = curIo.getNextMetaPageId(curPageAddr);
                         }
                         else
                             tailIdx += written;
@@ -253,25 +253,25 @@ public abstract class PagesList extends DataStructure {
             }
         }
         finally {
-            releaseAndClose(curPage, curBuf);
+            releaseAndClose(curPage, curPageAddr);
         }
 
         while (nextPageId != 0L) {
             try (Page page = page(nextPageId)) {
-                ByteBuffer buf = writeLock(page);
+                long pageAddr = writeLock(page);
 
                 try {
-                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+                    PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-                    io.resetCount(buf);
+                    io.resetCount(pageAddr);
 
                     if (PageHandler.isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PageListMetaResetCountRecord(cacheId, nextPageId));
 
-                    nextPageId = io.getNextMetaPageId(buf);
+                    nextPageId = io.getNextMetaPageId(pageAddr);
                 }
                 finally {
-                    writeUnlock(page, buf, true);
+                    writeUnlock(page, pageAddr, true);
                 }
             }
         }
@@ -281,7 +281,7 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param buf Buffer.
      */
-    private void releaseAndClose(Page page, ByteBuffer buf) {
+    private void releaseAndClose(Page page, long buf) {
         if (page != null) {
             try {
                 // No special WAL record because we most likely changed the whole page.
@@ -322,10 +322,10 @@ public abstract class PagesList extends DataStructure {
      * @param nextId Next page ID.
      * @param next Next page buffer.
      */
-    private void setupNextPage(PagesListNodeIO io, long prevId, ByteBuffer prev, long nextId, ByteBuffer next) {
+    private void setupNextPage(PagesListNodeIO io, long prevId, long prev, long nextId, long next) {
         assert io.getNextId(prev) == 0L;
 
-        io.initNewPage(next, nextId);
+        io.initNewPage(next, nextId, pageSize());
         io.setPreviousId(next, prevId);
 
         io.setNextId(prev, nextId);
@@ -342,7 +342,7 @@ public abstract class PagesList extends DataStructure {
         long pageId = reuse ? allocatePage(null) : allocatePageNoReuse();
 
         try (Page page = page(pageId)) {
-            initPage(page, this, PagesListNodeIO.VERSIONS.latest(), wal);
+            initPage(pageMem, page, this, PagesListNodeIO.VERSIONS.latest(), wal);
         }
 
         Stripe stripe = new Stripe(pageId);
@@ -466,19 +466,19 @@ public abstract class PagesList extends DataStructure {
                 long pageId = tail.tailId;
 
                 try (Page page = page(pageId)) {
-                    ByteBuffer buf = readLock(page); // No correctness guaranties.
+                    long pageAddr = readLock(page); // No correctness guaranties.
 
                     try {
-                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                        int cnt = io.getCount(buf);
+                        int cnt = io.getCount(pageAddr);
 
                         assert cnt >= 0;
 
                         res += cnt;
                     }
                     finally {
-                        readUnlock(page, buf);
+                        readUnlock(page, pageAddr);
                     }
                 }
             }
@@ -493,9 +493,9 @@ public abstract class PagesList extends DataStructure {
      * @param bucket Bucket.
      * @throws IgniteCheckedException If failed.
      */
-    protected final void put(ReuseBag bag, Page dataPage, ByteBuffer dataPageBuf, int bucket)
+    protected final void put(ReuseBag bag, Page dataPage, long dataPageBuf, int bucket)
         throws IgniteCheckedException {
-        assert bag == null ^ dataPageBuf == null;
+        assert bag == null ^ dataPageBuf == 0L;
 
         for (int lockAttempt = 0; ;) {
             Stripe stripe = getPageForPut(bucket);
@@ -503,9 +503,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long buf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (buf == null)
+                if (buf == 0L)
                     continue;
 
                 assert PageIO.getPageId(buf) == tailId : "bufPageId = " + PageIO.getPageId(buf) + ", tailId = " + tailId;
@@ -551,10 +551,10 @@ public abstract class PagesList extends DataStructure {
     private boolean putDataPage(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         if (io.getNextId(buf) != 0L)
@@ -562,7 +562,7 @@ public abstract class PagesList extends DataStructure {
 
         long dataPageId = dataPage.id();
 
-        int idx = io.addPage(buf, dataPageId);
+        int idx = io.addPage(buf, dataPageId, pageSize());
 
         if (idx == -1)
             handlePageFull(pageId, page, buf, io, dataPage, dataPageBuf, bucket);
@@ -593,10 +593,10 @@ public abstract class PagesList extends DataStructure {
     private void handlePageFull(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page dataPage,
-        ByteBuffer dataPageBuf,
+        long dataPageBuf,
         int bucket
     ) throws IgniteCheckedException {
         long dataPageId = dataPage.id();
@@ -631,17 +631,17 @@ public abstract class PagesList extends DataStructure {
             long nextId = allocatePage(null);
 
             try (Page next = page(nextId)) {
-                ByteBuffer nextBuf = writeLock(next); // Newly allocated page.
+                long nextPageAddr = writeLock(next); // Newly allocated page.
 
-                assert nextBuf != null;
+                assert nextPageAddr != 0L;
 
                 try {
-                    setupNextPage(io, pageId, buf, nextId, nextBuf);
+                    setupNextPage(io, pageId, buf, nextId, nextPageAddr);
 
                     if (isWalDeltaRecordNeeded(wal, page))
                         wal.log(new PagesListSetNextRecord(cacheId, pageId, nextId));
 
-                    int idx = io.addPage(nextBuf, dataPageId);
+                    int idx = io.addPage(nextPageAddr, dataPageId, pageSize());
 
                     // Here we should never write full page, because it is known to be new.
                     next.fullPageWalRecordPolicy(FALSE);
@@ -667,7 +667,7 @@ public abstract class PagesList extends DataStructure {
                     updateTail(bucket, pageId, nextId);
                 }
                 finally {
-                    writeUnlock(next, nextBuf, true);
+                    writeUnlock(next, nextPageAddr, true);
                 }
             }
         }
@@ -687,7 +687,7 @@ public abstract class PagesList extends DataStructure {
     private boolean putReuseBag(
         final long pageId,
         Page page,
-        final ByteBuffer buf,
+        final long buf,
         PagesListNodeIO io,
         ReuseBag bag,
         int bucket
@@ -696,21 +696,21 @@ public abstract class PagesList extends DataStructure {
             return false; // Splitted.
 
         long nextId;
-        ByteBuffer prevBuf = buf;
+        long prevBuf = buf;
         long prevId = pageId;
 
         List<Page> locked = null; // TODO may be unlock right away and do not keep all these pages locked?
-        List<ByteBuffer> lockedBufs = null;
+        List<Long> lockedBufs = null;
 
         try {
             while ((nextId = bag.pollFreePage()) != 0L) {
-                int idx = io.addPage(prevBuf, nextId);
+                int idx = io.addPage(prevBuf, nextId, pageSize());
 
                 if (idx == -1) { // Attempt to add page failed: the node page is full.
                     try (Page next = page(nextId)) {
-                        ByteBuffer nextBuf = writeLock(next); // Page from reuse bag can't be concurrently recycled.
+                        long nextPageAddr = writeLock(next); // Page from reuse bag can't be concurrently recycled.
 
-                        assert nextBuf != null;
+                        assert nextPageAddr != 0L;
 
                         if (locked == null) {
                             lockedBufs = new ArrayList<>(2);
@@ -718,9 +718,9 @@ public abstract class PagesList extends DataStructure {
                         }
 
                         locked.add(next);
-                        lockedBufs.add(nextBuf);
+                        lockedBufs.add(nextPageAddr);
 
-                        setupNextPage(io, prevId, prevBuf, nextId, nextBuf);
+                        setupNextPage(io, prevId, prevBuf, nextId, nextPageAddr);
 
                         if (isWalDeltaRecordNeeded(wal, page))
                             wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -741,7 +741,7 @@ public abstract class PagesList extends DataStructure {
 
                         // Switch to this new page, which is now a part of our list
                         // to add the rest of the bag to the new page.
-                        prevBuf = nextBuf;
+                        prevBuf = nextPageAddr;
                         prevId = nextId;
                         page = next;
                     }
@@ -784,15 +784,15 @@ public abstract class PagesList extends DataStructure {
      * @param page Page.
      * @param bucket Bucket.
      * @param lockAttempt Lock attempts counter.
-     * @return Buffer if page is locket of {@code null} if can retry lock.
+     * @return Page address if page is locked of {@code null} if can retry lock.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private ByteBuffer writeLockPage(Page page, int bucket, int lockAttempt)
+    private long writeLockPage(Page page, int bucket, int lockAttempt)
         throws IgniteCheckedException {
-        ByteBuffer buf = tryWriteLock(page);
+        long pageAddr = tryWriteLock(page);
 
-        if (buf != null)
-            return buf;
+        if (pageAddr != 0L)
+            return pageAddr;
 
         if (lockAttempt == TRY_LOCK_ATTEMPTS) {
             Stripe[] stripes = getBucket(bucket);
@@ -800,11 +800,11 @@ public abstract class PagesList extends DataStructure {
             if (stripes == null || stripes.length < MAX_STRIPES_PER_BUCKET) {
                 addStripe(bucket, false);
 
-                return null;
+                return 0L;
             }
         }
 
-        return lockAttempt < TRY_LOCK_ATTEMPTS ? null : writeLock(page); // Must be explicitly checked further.
+        return lockAttempt < TRY_LOCK_ATTEMPTS ? 0L : writeLock(page); // Must be explicitly checked further.
     }
 
     /**
@@ -823,9 +823,9 @@ public abstract class PagesList extends DataStructure {
             long tailId = stripe.tailId;
 
             try (Page tail = page(tailId)) {
-                ByteBuffer tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
+                long tailBuf = writeLockPage(tail, bucket, lockAttempt++); // Explicit check.
 
-                if (tailBuf == null)
+                if (tailBuf == 0L)
                     continue;
 
                 assert PageIO.getPageId(tailBuf) == tailId : "tailId = " + tailId + ", tailBufId = " + PageIO.getPageId(tailBuf);
@@ -859,7 +859,7 @@ public abstract class PagesList extends DataStructure {
                             if (prevId != 0L) {
                                 try (Page prev = page(prevId)) {
                                     // Lock pages from next to previous.
-                                    Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                    Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                     assert ok == TRUE : ok;
                                 }
@@ -878,7 +878,7 @@ public abstract class PagesList extends DataStructure {
 
                             try (Page prev = page(prevId)) {
                                 // Lock pages from next to previous.
-                                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                                 assert ok == TRUE : ok;
                             }
@@ -888,7 +888,7 @@ public abstract class PagesList extends DataStructure {
 
                                 PageIO initIo = initIoVers.latest();
 
-                                initIo.initNewPage(tailBuf, tailId);
+                                initIo.initNewPage(tailBuf, tailId, pageSize());
 
                                 if (isWalDeltaRecordNeeded(wal, tail)) {
                                     wal.log(new InitNewPageRecord(cacheId, tail.id(), initIo.getType(),
@@ -933,7 +933,7 @@ public abstract class PagesList extends DataStructure {
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if page was removed.
      */
-    protected final boolean removeDataPage(Page dataPage, ByteBuffer dataPageBuf, DataPageIO dataIO, int bucket)
+    protected final boolean removeDataPage(Page dataPage, long dataPageBuf, DataPageIO dataIO, int bucket)
         throws IgniteCheckedException {
         long dataPageId = dataPage.id();
 
@@ -947,17 +947,17 @@ public abstract class PagesList extends DataStructure {
 
             long recycleId = 0L;
 
-            ByteBuffer buf = writeLock(page); // Explicit check.
+            long pageAddr = writeLock(page); // Explicit check.
 
-            if (buf == null)
+            if (pageAddr == 0L)
                 return false;
 
             boolean rmvd = false;
 
             try {
-                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                rmvd = io.removePage(buf, dataPageId);
+                rmvd = io.removePage(pageAddr, dataPageId);
 
                 if (!rmvd)
                     return false;
@@ -971,20 +971,20 @@ public abstract class PagesList extends DataStructure {
                 if (isWalDeltaRecordNeeded(wal, dataPage))
                     wal.log(new DataPageSetFreeListPageRecord(cacheId, dataPageId, 0L));
 
-                if (!io.isEmpty(buf))
+                if (!io.isEmpty(pageAddr))
                     return true; // In optimistic case we still have something in the page and can leave it as is.
 
                 // If the page is empty, we have to try to drop it and link next and previous with each other.
-                nextId = io.getNextId(buf);
-                prevId = io.getPreviousId(buf);
+                nextId = io.getNextId(pageAddr);
+                prevId = io.getPreviousId(pageAddr);
 
                 // If there are no next page, then we can try to merge without releasing current write lock,
                 // because if we will need to lock previous page, the locking order will be already correct.
                 if (nextId == 0L)
-                    recycleId = mergeNoNext(pageId, page, buf, prevId, bucket);
+                    recycleId = mergeNoNext(pageId, page, pageAddr, prevId, bucket);
             }
             finally {
-                writeUnlock(page, buf, rmvd);
+                writeUnlock(page, pageAddr, rmvd);
             }
 
             // Perform a fair merge after lock release (to have a correct locking order).
@@ -1007,7 +1007,7 @@ public abstract class PagesList extends DataStructure {
      * @return Page ID to recycle.
      * @throws IgniteCheckedException If failed.
      */
-    private long mergeNoNext(long pageId, Page page, ByteBuffer buf, long prevId, int bucket)
+    private long mergeNoNext(long pageId, Page page, long buf, long prevId, int bucket)
         throws IgniteCheckedException {
         // If we do not have a next page (we are tail) and we are on reuse bucket,
         // then we can leave as is as well, because it is normal to have an empty tail page here.
@@ -1016,7 +1016,7 @@ public abstract class PagesList extends DataStructure {
 
         if (prevId != 0L) { // Cut tail if we have a previous page.
             try (Page prev = page(prevId)) {
-                Boolean ok = writePage(prev, this, cutTail, null, bucket, FALSE);
+                Boolean ok = writePage(pageMem, prev, this, cutTail, null, bucket, FALSE);
 
                 assert ok == TRUE: ok; // Because we keep lock on current tail and do a world consistency check.
             }
@@ -1044,25 +1044,25 @@ public abstract class PagesList extends DataStructure {
             try (Page next = nextId == 0L ? null : page(nextId)) {
                 boolean write = false;
 
-                ByteBuffer nextBuf = next == null ? null : writeLock(next); // Explicit check.
-                ByteBuffer buf = writeLock(page); // Explicit check.
+                long nextPageAddr = next == null ? 0L : writeLock(next); // Explicit check.
+                long pageAddr = writeLock(page); // Explicit check.
 
-                if (buf == null) {
-                    if (nextBuf != null) // Unlock next page if needed.
-                        writeUnlock(next, nextBuf, false);
+                if (pageAddr == 0L) {
+                    if (nextPageAddr != 0L) // Unlock next page if needed.
+                        writeUnlock(next, nextPageAddr, false);
 
                     return 0L; // Someone has merged or taken our empty page concurrently. Nothing to do here.
                 }
 
                 try {
-                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+                    PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-                    if (!io.isEmpty(buf))
+                    if (!io.isEmpty(pageAddr))
                         return 0L; // No need to merge anymore.
 
                     // Check if we see a consistent state of the world.
-                    if (io.getNextId(buf) == nextId && (nextId == 0L) == (nextBuf == null)) {
-                        long recycleId = doMerge(pageId, page, buf, io, next, nextId, nextBuf, bucket);
+                    if (io.getNextId(pageAddr) == nextId && (nextId == 0L) == (nextPageAddr == 0L)) {
+                        long recycleId = doMerge(pageId, page, pageAddr, io, next, nextId, nextPageAddr, bucket);
 
                         write = true;
 
@@ -1070,13 +1070,13 @@ public abstract class PagesList extends DataStructure {
                     }
 
                     // Reread next page ID and go for retry.
-                    nextId = io.getNextId(buf);
+                    nextId = io.getNextId(pageAddr);
                 }
                 finally {
-                    if (nextBuf != null)
-                        writeUnlock(next, nextBuf, write);
+                    if (nextPageAddr != 0L)
+                        writeUnlock(next, nextPageAddr, write);
 
-                    writeUnlock(page, buf, write);
+                    writeUnlock(page, pageAddr, write);
                 }
             }
         }
@@ -1097,11 +1097,11 @@ public abstract class PagesList extends DataStructure {
     private long doMerge(
         long pageId,
         Page page,
-        ByteBuffer buf,
+        long buf,
         PagesListNodeIO io,
         Page next,
         long nextId,
-        ByteBuffer nextBuf,
+        long nextBuf,
         int bucket
     ) throws IgniteCheckedException {
         long prevId = io.getPreviousId(buf);
@@ -1143,22 +1143,22 @@ public abstract class PagesList extends DataStructure {
         long pageId,
         long nextId,
         Page next,
-        ByteBuffer nextBuf)
+        long nextBuf)
         throws IgniteCheckedException {
         try (Page prev = page(prevId)) {
-            ByteBuffer prevBuf = writeLock(prev); // No check, we keep a reference.
+            long prevPageAddr = writeLock(prev); // No check, we keep a reference.
 
-            assert prevBuf != null;
+            assert prevPageAddr != 0L;
 
             try {
-                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevBuf);
+                PagesListNodeIO prevIO = PagesListNodeIO.VERSIONS.forPage(prevPageAddr);
                 PagesListNodeIO nextIO = PagesListNodeIO.VERSIONS.forPage(nextBuf);
 
                 // These references must be updated at the same time in write locks.
-                assert prevIO.getNextId(prevBuf) == pageId;
+                assert prevIO.getNextId(prevPageAddr) == pageId;
                 assert nextIO.getPreviousId(nextBuf) == pageId;
 
-                prevIO.setNextId(prevBuf, nextId);
+                prevIO.setNextId(prevPageAddr, nextId);
 
                 if (isWalDeltaRecordNeeded(wal, prev))
                     wal.log(new PagesListSetNextRecord(cacheId, prevId, nextId));
@@ -1169,7 +1169,7 @@ public abstract class PagesList extends DataStructure {
                     wal.log(new PagesListSetPreviousRecord(cacheId, nextId, prevId));
             }
             finally {
-                writeUnlock(prev, prevBuf, true);
+                writeUnlock(prev, prevPageAddr, true);
             }
         }
     }
@@ -1181,7 +1181,7 @@ public abstract class PagesList extends DataStructure {
      * @return Rotated page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long recyclePage(long pageId, Page page, ByteBuffer buf) throws IgniteCheckedException {
+    private long recyclePage(long pageId, Page page, long buf) throws IgniteCheckedException {
         pageId = PageIdUtils.rotatePageId(pageId);
 
         PageIO.setPageId(buf, pageId);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
index b525a01..6bd50a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListMetaIO.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
 import java.util.Map;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.freelist.PagesList;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
@@ -53,66 +53,67 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setCount(buf, 0);
-        setNextMetaPageId(buf, 0L);
+        setCount(pageAddr, 0);
+        setNextMetaPageId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Stored items count.
      */
-    private int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    private int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
-     * @param buf Buffer,
+     * @param pageAddr Page address.
      * @param cnt Stored items count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next meta page ID.
      */
-    public long getNextMetaPageId(ByteBuffer buf) {
-        return buf.getLong(NEXT_META_PAGE_OFF);
+    public long getNextMetaPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_META_PAGE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param metaPageId Next meta page ID.
      */
-    public void setNextMetaPageId(ByteBuffer buf, long metaPageId) {
-        buf.putLong(NEXT_META_PAGE_OFF, metaPageId);
+    public void setNextMetaPageId(long pageAddr, long metaPageId) {
+        PageUtils.putLong(pageAddr, NEXT_META_PAGE_OFF, metaPageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void resetCount(ByteBuffer buf) {
-        setCount(buf, 0);
+    public void resetCount(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageSize Page size.
+     * @param pageAddr Page address.
      * @param bucket Bucket number.
      * @param tails Tails.
      * @param tailsOff Tails offset.
      * @return Number of items written.
      */
-    public int addTails(ByteBuffer buf, int bucket, PagesList.Stripe[] tails, int tailsOff) {
+    public int addTails(int pageSize, long pageAddr, int bucket, PagesList.Stripe[] tails, int tailsOff) {
         assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-        int cnt = getCount(buf);
-        int cap = getCapacity(buf);
+        int cnt = getCount(pageAddr);
+        int cap = getCapacity(pageSize, pageAddr);
 
         if (cnt == cap)
             return 0;
@@ -122,25 +123,25 @@ public class PagesListMetaIO extends PageIO {
         int write = Math.min(cap - cnt, tails.length - tailsOff);
 
         for (int i = 0; i < write; i++) {
-            buf.putShort(off, (short)bucket);
-            buf.putLong(off + 2, tails[tailsOff].tailId);
+            PageUtils.putShort(pageAddr, off, (short)bucket);
+            PageUtils.putLong(pageAddr, off + 2, tails[tailsOff].tailId);
 
             tailsOff++;
 
             off += ITEM_SIZE;
         }
 
-        setCount(buf, cnt + write);
+        setCount(pageAddr, cnt + write);
 
         return write;
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param res Results map.
      */
-    public void getBucketsData(ByteBuffer buf, Map<Integer, GridLongList> res) {
-        int cnt = getCount(buf);
+    public void getBucketsData(long pageAddr, Map<Integer, GridLongList> res) {
+        int cnt = getCount(pageAddr);
 
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
@@ -150,10 +151,10 @@ public class PagesListMetaIO extends PageIO {
         int off = offset(0);
 
         for (int i = 0; i < cnt; i++) {
-            Integer bucket = (int)buf.getShort(off);
+            Integer bucket = (int)PageUtils.getShort(pageAddr, off);
             assert bucket >= 0 && bucket <= Short.MAX_VALUE : bucket;
 
-            long tailId = buf.getLong(off + 2);
+            long tailId = PageUtils.getLong(pageAddr, off + 2);
             assert tailId != 0;
 
             GridLongList list = res.get(bucket);
@@ -168,11 +169,11 @@ public class PagesListMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Maximum number of items which can be stored in buffer.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - ITEMS_OFF) / ITEM_SIZE;
+    private int getCapacity(int pageSize, long pageAddr) {
+        return (pageSize - ITEMS_OFF) / ITEM_SIZE;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
index 2349fa1..6bd0532 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/freelist/io/PagesListNodeIO.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache.database.freelist.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.IOVersions;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
 import static org.apache.ignite.internal.processors.cache.database.tree.util.PageHandler.copyMemory;
 
 /**
- * TODO optimize: now we have slow {@link #removePage(ByteBuffer, long)}
+ * TODO optimize: now we have slow {@link #removePage(long, long)}
  */
 public class PagesListNodeIO extends PageIO {
     /** */
@@ -52,84 +52,84 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setEmpty(buf);
+        setEmpty(pageAddr);
 
-        setPreviousId(buf, 0L);
-        setNextId(buf, 0L);
+        setPreviousId(pageAddr, 0L);
+        setNextId(pageAddr, 0L);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    private void setEmpty(ByteBuffer buf) {
-        setCount(buf, 0);
+    private void setEmpty(long pageAddr) {
+        setCount(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Next page ID.
      */
-    public long getNextId(ByteBuffer buf) {
-        return buf.getLong(NEXT_PAGE_ID_OFF);
+    public long getNextId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextId Next page ID.
      */
-    public void setNextId(ByteBuffer buf, long nextId) {
-        buf.putLong(NEXT_PAGE_ID_OFF, nextId);
+    public void setNextId(long pageAddr, long nextId) {
+        PageUtils.putLong(pageAddr, NEXT_PAGE_ID_OFF, nextId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Previous page ID.
      */
-    public long getPreviousId(ByteBuffer buf) {
-        return buf.getLong(PREV_PAGE_ID_OFF);
+    public long getPreviousId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PREV_PAGE_ID_OFF);
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param prevId Previous  page ID.
      */
-    public void setPreviousId(ByteBuffer buf, long prevId) {
-        buf.putLong(PREV_PAGE_ID_OFF, prevId);
+    public void setPreviousId(long pageAddr, long prevId) {
+        PageUtils.putLong(pageAddr, PREV_PAGE_ID_OFF, prevId);
     }
 
     /**
      * Gets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to get count from.
+     * @param pageAddr Page address to get count from.
      * @return Total number of entries.
      */
-    public int getCount(ByteBuffer buf) {
-        return buf.getShort(CNT_OFF);
+    public int getCount(long pageAddr) {
+        return PageUtils.getShort(pageAddr, CNT_OFF);
     }
 
     /**
      * Sets total count of entries in this page. Does not change the buffer state.
      *
-     * @param buf Page buffer to write to.
+     * @param pageAddr Page address to write to.
      * @param cnt Count.
      */
-    private void setCount(ByteBuffer buf, int cnt) {
+    private void setCount(long pageAddr, int cnt) {
         assert cnt >= 0 && cnt <= Short.MAX_VALUE : cnt;
 
-        buf.putShort(CNT_OFF, (short)cnt);
+        PageUtils.putShort(pageAddr, CNT_OFF, (short)cnt);
     }
 
     /**
      * Gets capacity of this page in items.
      *
-     * @param buf Page buffer to get capacity.
+     * @param pageSize Page size.
      * @return Capacity of this page in items.
      */
-    private int getCapacity(ByteBuffer buf) {
-        return (buf.capacity() - PAGE_IDS_OFF) >>> 3; // /8
+    private int getCapacity(int pageSize) {
+        return (pageSize - PAGE_IDS_OFF) >>> 3; // /8
     }
 
     /**
@@ -141,38 +141,39 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @return Item at the given index.
      */
-    private long getAt(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    private long getAt(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Item index.
      * @param pageId Item value to write.
      */
-    private void setAt(ByteBuffer buf, int idx, long pageId) {
-        buf.putLong(offset(idx), pageId);
+    private void setAt(long pageAddr, int idx, long pageId) {
+        PageUtils.putLong(pageAddr, offset(idx), pageId);
     }
 
     /**
      * Adds page to the end of pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      * @return Total number of items in this page.
      */
-    public int addPage(ByteBuffer buf, long pageId) {
-        int cnt = getCount(buf);
+    public int addPage(long pageAddr, long pageId, int pageSize) {
+        int cnt = getCount(pageAddr);
 
-        if (cnt == getCapacity(buf))
+        if (cnt == getCapacity(pageSize))
             return -1;
 
-        setAt(buf, cnt, pageId);
-        setCount(buf, cnt + 1);
+        setAt(pageAddr, cnt, pageId);
+        setCount(pageAddr, cnt + 1);
 
         return cnt;
     }
@@ -180,38 +181,38 @@ public class PagesListNodeIO extends PageIO {
     /**
      * Removes any page from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @return Removed page ID.
      */
-    public long takeAnyPage(ByteBuffer buf) {
-        int cnt = getCount(buf);
+    public long takeAnyPage(long pageAddr) {
+        int cnt = getCount(pageAddr);
 
         if (cnt == 0)
             return 0L;
 
-        setCount(buf, --cnt);
+        setCount(pageAddr, --cnt);
 
-        return getAt(buf, cnt);
+        return getAt(pageAddr, cnt);
     }
 
     /**
      * Removes the given page ID from the pages list.
      *
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param dataPageId Page ID to remove.
      * @return {@code true} if page was in the list and was removed, {@code false} otherwise.
      */
-    public boolean removePage(ByteBuffer buf, long dataPageId) {
+    public boolean removePage(long pageAddr, long dataPageId) {
         assert dataPageId != 0;
 
-        int cnt = getCount(buf);
+        int cnt = getCount(pageAddr);
 
         for (int i = 0; i < cnt; i++) {
-            if (getAt(buf, i) == dataPageId) {
+            if (getAt(pageAddr, i) == dataPageId) {
                 if (i != cnt - 1)
-                    copyMemory(buf, buf, offset(i + 1), offset(i), 8 * (cnt - i - 1));
+                    copyMemory(pageAddr, pageAddr, offset(i + 1), offset(i), 8 * (cnt - i - 1));
 
-                setCount(buf, cnt - 1);
+                setCount(pageAddr, cnt - 1);
 
                 return true;
             }
@@ -221,10 +222,10 @@ public class PagesListNodeIO extends PageIO {
     }
 
     /**
-     * @param buf Page buffer.
-     * @return {@code true} if there are no items in this page.
+     * @param pageAddr Page address.
+     * @return {@code True} if there are no items in this page.
      */
-    public boolean isEmpty(ByteBuffer buf) {
-        return getCount(buf) == 0;
+    public boolean isEmpty(long pageAddr) {
+        return getCount(pageAddr) == 0;
     }
 }


[35/50] [abbrv] ignite git commit: IGNITE-4045 .NET: Support DML API

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
index 08789b6..ce9fcf6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs
@@ -157,8 +157,6 @@
         {
             IgniteArgumentCheck.NotNull(configuration, "configuration");
 
-            CopyLocalProperties(configuration);
-
             using (var stream = IgniteManager.Memory.Allocate().GetStream())
             {
                 var marsh = new Marshaller(configuration.BinaryConfiguration);
@@ -171,6 +169,8 @@
 
                 ReadCore(marsh.StartUnmarshal(stream));
             }
+
+            CopyLocalProperties(configuration);
         }
 
         /// <summary>
@@ -255,12 +255,37 @@
                 writer.WriteBoolean(false);
 
             // Binary config
-            var isCompactFooterSet = BinaryConfiguration != null && BinaryConfiguration.CompactFooterInternal != null;
+            if (BinaryConfiguration != null)
+            {
+                writer.WriteBoolean(true);
 
-            writer.WriteBoolean(isCompactFooterSet);
+                if (BinaryConfiguration.CompactFooterInternal != null)
+                {
+                    writer.WriteBoolean(true);
+                    writer.WriteBoolean(BinaryConfiguration.CompactFooter);
+                }
+                else
+                {
+                    writer.WriteBoolean(false);
+                }
+
+                // Send only descriptors with non-null EqualityComparer to preserve old behavior where
+                // remote nodes can have no BinaryConfiguration.
+                var types = writer.Marshaller.GetUserTypeDescriptors().Where(x => x.EqualityComparer != null).ToList();
+
+                writer.WriteInt(types.Count);
 
-            if (isCompactFooterSet)
-                writer.WriteBoolean(BinaryConfiguration.CompactFooter);
+                foreach (var type in types)
+                {
+                    writer.WriteString(BinaryUtils.SimpleTypeName(type.TypeName));
+                    writer.WriteBoolean(type.IsEnum);
+                    BinaryEqualityComparerSerializer.Write(writer, type.EqualityComparer);
+                }
+            }
+            else
+            {
+                writer.WriteBoolean(false);
+            }
 
             // User attributes
             var attrs = UserAttributes;
@@ -361,7 +386,28 @@
             if (r.ReadBoolean())
             {
                 BinaryConfiguration = BinaryConfiguration ?? new BinaryConfiguration();
-                BinaryConfiguration.CompactFooter = r.ReadBoolean();
+
+                if (r.ReadBoolean())
+                    BinaryConfiguration.CompactFooter = r.ReadBoolean();
+
+                var typeCount = r.ReadInt();
+
+                if (typeCount > 0)
+                {
+                    var types = new List<BinaryTypeConfiguration>(typeCount);
+
+                    for (var i = 0; i < typeCount; i++)
+                    {
+                        types.Add(new BinaryTypeConfiguration
+                        {
+                            TypeName = r.ReadString(),
+                            IsEnum = r.ReadBoolean(),
+                            EqualityComparer = BinaryEqualityComparerSerializer.Read(r)
+                        });
+                    }
+
+                    BinaryConfiguration.TypeConfigurations = types;
+                }
             }
 
             // User attributes
@@ -402,17 +448,15 @@
         /// <param name="binaryReader">The binary reader.</param>
         private void Read(BinaryReader binaryReader)
         {
-            var r = binaryReader;
-
-            CopyLocalProperties(r.Marshaller.Ignite.Configuration);
+            ReadCore(binaryReader);
 
-            ReadCore(r);
+            CopyLocalProperties(binaryReader.Marshaller.Ignite.Configuration);
 
             // Misc
-            IgniteHome = r.ReadString();
+            IgniteHome = binaryReader.ReadString();
 
-            JvmInitialMemoryMb = (int) (r.ReadLong()/1024/2014);
-            JvmMaxMemoryMb = (int) (r.ReadLong()/1024/2014);
+            JvmInitialMemoryMb = (int) (binaryReader.ReadLong()/1024/2014);
+            JvmMaxMemoryMb = (int) (binaryReader.ReadLong()/1024/2014);
 
             // Local data (not from reader)
             JvmDllPath = Process.GetCurrentProcess().Modules.OfType<ProcessModule>()
@@ -426,9 +470,16 @@
         private void CopyLocalProperties(IgniteConfiguration cfg)
         {
             GridName = cfg.GridName;
-            BinaryConfiguration = cfg.BinaryConfiguration == null
-                ? null
-                : new BinaryConfiguration(cfg.BinaryConfiguration);
+
+            if (BinaryConfiguration != null && cfg.BinaryConfiguration != null)
+            {
+                BinaryConfiguration.MergeTypes(cfg.BinaryConfiguration);
+            }
+            else if (cfg.BinaryConfiguration != null)
+            {
+                BinaryConfiguration = new BinaryConfiguration(cfg.BinaryConfiguration);
+            }
+            
             JvmClasspath = cfg.JvmClasspath;
             JvmOptions = cfg.JvmOptions;
             Assemblies = cfg.Assemblies;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index b35527d..d54a200 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -152,6 +152,20 @@
                                                             </xs:attribute>
                                                         </xs:complexType>
                                                     </xs:element>
+                                                    <xs:element name="equalityComparer" minOccurs="0">
+                                                        <xs:annotation>
+                                                            <xs:documentation>
+                                                                Equality comparer to compute hash codes and compare objects in IBinaryObject form.
+                                                            </xs:documentation>
+                                                        </xs:annotation>
+                                                        <xs:complexType>
+                                                            <xs:attribute name="type" type="xs:string" use="required">
+                                                                <xs:annotation>
+                                                                    <xs:documentation>Assembly-qualified type name.</xs:documentation>
+                                                                </xs:annotation>
+                                                            </xs:attribute>
+                                                        </xs:complexType>
+                                                    </xs:element>
                                                 </xs:all>
                                                 <xs:attribute name="typeName" type="xs:string">
                                                     <xs:annotation>
@@ -297,6 +311,11 @@
                                                                                             <xs:documentation>Java field type name.</xs:documentation>
                                                                                         </xs:annotation>
                                                                                     </xs:attribute>
+                                                                                    <xs:attribute name="isKeyField" type="xs:boolean">
+                                                                                        <xs:annotation>
+                                                                                            <xs:documentation>Indicates whether this field belongs to the cache key.</xs:documentation>
+                                                                                        </xs:annotation>
+                                                                                    </xs:attribute>
                                                                                 </xs:complexType>
                                                                             </xs:element>
                                                                         </xs:sequence>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
index 3d55acd..fa7cf6c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs
@@ -81,7 +81,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 throw new IgniteException("Type is not binary (add it to BinaryConfiguration): " + 
                     type.FullName);
 
-            return Builder0(null, BinaryFromDescriptor(desc), desc);
+            return Builder0(null, null, desc);
         }
 
         /** <inheritDoc /> */
@@ -91,7 +91,7 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             IBinaryTypeDescriptor desc = _marsh.GetDescriptor(typeName);
             
-            return Builder0(null, BinaryFromDescriptor(desc), desc);
+            return Builder0(null, null, desc);
         }
 
         /** <inheritDoc /> */
@@ -182,30 +182,6 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
-        /// Create empty binary object from descriptor.
-        /// </summary>
-        /// <param name="desc">Descriptor.</param>
-        /// <returns>Empty binary object.</returns>
-        private BinaryObject BinaryFromDescriptor(IBinaryTypeDescriptor desc)
-        {
-            const int len = BinaryObjectHeader.Size;
-
-            var flags = desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None;
-
-            if (_marsh.CompactFooter && desc.UserType)
-                flags |= BinaryObjectHeader.Flag.CompactFooter;
-
-            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len, flags);
-
-            using (var stream = new BinaryHeapStream(len))
-            {
-                BinaryObjectHeader.Write(hdr, stream, 0);
-
-                return new BinaryObject(_marsh, stream.InternalArray, 0, hdr);
-            }
-        }
-
-        /// <summary>
         /// Internal builder creation routine.
         /// </summary>
         /// <param name="parent">Parent builder.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
new file mode 100644
index 0000000..aa4795e
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryEqualityComparerSerializer.cs
@@ -0,0 +1,99 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Reads and writes <see cref="IBinaryEqualityComparer"/>.
+    /// </summary>
+    internal static class BinaryEqualityComparerSerializer
+    {
+        /// <summary>
+        /// SwapSpace type.
+        /// </summary>
+        private enum Type : byte
+        {
+            None = 0,
+            Array = 1,
+            Field = 2
+        }
+
+        /// <summary>
+        /// Writes an instance.
+        /// </summary>
+        public static void Write(IBinaryRawWriter writer, IBinaryEqualityComparer comparer)
+        {
+            if (comparer == null)
+            {
+                writer.WriteByte((byte) Type.None);
+                return;
+            }
+
+            var arrCmp = comparer as BinaryArrayEqualityComparer;
+
+            if (arrCmp != null)
+            {
+                writer.WriteByte((byte) Type.Array);
+                return;
+            }
+
+            var fieldCmp = (BinaryFieldEqualityComparer) comparer;
+
+            writer.WriteByte((byte) Type.Field);
+
+            fieldCmp.Validate();
+
+            writer.WriteInt(fieldCmp.FieldNames.Count);
+
+            foreach (var field in fieldCmp.FieldNames)
+                writer.WriteString(field);
+        }
+
+        /// <summary>
+        /// Reads an instance.
+        /// </summary>
+        /// <param name="reader">The reader.</param>
+        /// <returns></returns>
+        public static IEqualityComparer<IBinaryObject> Read(IBinaryRawReader reader)
+        {
+            var type = (Type) reader.ReadByte();
+
+            switch (type)
+            {
+                case Type.None:
+                    return null;
+
+                case Type.Array:
+                    return new BinaryArrayEqualityComparer();
+
+                case Type.Field:
+                    return new BinaryFieldEqualityComparer
+                    {
+                        FieldNames = Enumerable.Range(0, reader.ReadInt()).Select(x => reader.ReadString()).ToArray()
+                    };
+
+                default:
+                    throw new ArgumentOutOfRangeException();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFieldEqualityComparer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFieldEqualityComparer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFieldEqualityComparer.cs
new file mode 100644
index 0000000..433657a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFieldEqualityComparer.cs
@@ -0,0 +1,138 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using System.IO;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Common;
+
+    /// <summary>
+    /// Uses a set of binary object fields to calculate hash code and check equality.
+    /// Not implemented for now, will be done as part of IGNITE-4397.
+    /// </summary>
+    internal class BinaryFieldEqualityComparer : IEqualityComparer<IBinaryObject>, IBinaryEqualityComparer
+    {
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryFieldEqualityComparer"/> class.
+        /// </summary>
+        public BinaryFieldEqualityComparer()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="BinaryFieldEqualityComparer"/> class.
+        /// </summary>
+        /// <param name="fieldNames">The field names for comparison.</param>
+        public BinaryFieldEqualityComparer(params string[] fieldNames)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(fieldNames, "fieldNames");
+
+            FieldNames = fieldNames;
+        }
+
+        /// <summary>
+        /// Gets or sets the field names to be used for equality comparison.
+        /// </summary>
+        public ICollection<string> FieldNames { get; set; }
+
+        /// <summary>
+        /// Determines whether the specified objects are equal.
+        /// </summary>
+        /// <param name="x">The first object to compare.</param>
+        /// <param name="y">The second object to compare.</param>
+        /// <returns>
+        /// true if the specified objects are equal; otherwise, false.
+        /// </returns>
+        public bool Equals(IBinaryObject x, IBinaryObject y)
+        {
+            throw new NotSupportedException(GetType() + "is not intended for direct usage.");
+        }
+
+        /// <summary>
+        /// Returns a hash code for this instance.
+        /// </summary>
+        /// <param name="obj">The object.</param>
+        /// <returns>
+        /// A hash code for this instance, suitable for use in hashing algorithms and data structures like a hash table. 
+        /// </returns>
+        public int GetHashCode(IBinaryObject obj)
+        {
+            throw new NotSupportedException(GetType() + "is not intended for direct usage.");
+        }
+
+        /** <inheritdoc /> */
+        int IBinaryEqualityComparer.GetHashCode(IBinaryStream stream, int startPos, int length,
+            BinaryObjectSchemaHolder schema, int schemaId, Marshaller marshaller, IBinaryTypeDescriptor desc)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(startPos >= 0);
+            Debug.Assert(length >= 0);
+            Debug.Assert(schema != null);
+            Debug.Assert(marshaller != null);
+            Debug.Assert(desc != null);
+
+            Validate();
+
+            stream.Flush();
+
+            // Preserve stream position.
+            var pos = stream.Position;
+
+            var reader = marshaller.StartUnmarshal(stream, BinaryMode.ForceBinary);
+            var fields = schema.GetFullSchema(schemaId);
+
+            int hash = 0;
+
+            foreach (var fieldName in FieldNames)
+            {
+                int fieldId = BinaryUtils.FieldId(desc.TypeId, fieldName, desc.NameMapper, desc.IdMapper);
+                int fieldHash = 0;  // Null (missing) field hash code is 0.
+                int fieldPos;
+
+                if (fields.TryGetValue(fieldId, out fieldPos))
+                {
+                    stream.Seek(startPos + fieldPos - BinaryObjectHeader.Size, SeekOrigin.Begin);
+                    var fieldVal = reader.Deserialize<object>();
+                    fieldHash = fieldVal != null ? fieldVal.GetHashCode() : 0;
+                }
+
+                hash = 31 * hash + fieldHash;
+            }
+
+            // Restore stream position.
+            stream.Seek(pos, SeekOrigin.Begin);
+
+            return hash;
+        }
+
+        /// <summary>
+        /// Validates this instance.
+        /// </summary>
+        public void Validate()
+        {
+            if (FieldNames == null || FieldNames.Count == 0)
+                throw new IgniteException("BinaryFieldEqualityComparer.FieldNames can not be null or empty.");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
index 6dc5d4d..d88e7a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryFullTypeDescriptor.cs
@@ -20,6 +20,7 @@ namespace Apache.Ignite.Core.Impl.Binary
     using System;
     using System.Collections.Generic;
     using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary.Structure;
 
     /// <summary>
@@ -66,6 +67,9 @@ namespace Apache.Ignite.Core.Impl.Binary
         /** Enum flag. */
         private readonly bool _isEnum;
 
+        /** Comparer. */
+        private readonly IBinaryEqualityComparer _equalityComparer;
+
         /// <summary>
         /// Constructor.
         /// </summary>
@@ -79,6 +83,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="keepDeserialized">Whether to cache deserialized value in IBinaryObject</param>
         /// <param name="affKeyFieldName">Affinity field key name.</param>
         /// <param name="isEnum">Enum flag.</param>
+        /// <param name="comparer">Equality comparer.</param>
         public BinaryFullTypeDescriptor(
             Type type, 
             int typeId, 
@@ -89,7 +94,8 @@ namespace Apache.Ignite.Core.Impl.Binary
             IBinarySerializerInternal serializer, 
             bool keepDeserialized, 
             string affKeyFieldName,
-            bool isEnum)
+            bool isEnum, 
+            IEqualityComparer<IBinaryObject> comparer)
         {
             _type = type;
             _typeId = typeId;
@@ -101,6 +107,13 @@ namespace Apache.Ignite.Core.Impl.Binary
             _keepDeserialized = keepDeserialized;
             _affKeyFieldName = affKeyFieldName;
             _isEnum = isEnum;
+
+            _equalityComparer = comparer as IBinaryEqualityComparer;
+
+            if (comparer != null && _equalityComparer == null)
+                throw new IgniteException(string.Format("Unsupported IEqualityComparer<IBinaryObject> " +
+                                                        "implementation: {0}. Only predefined implementations " +
+                                                        "are supported.", comparer.GetType()));
         }
 
         /// <summary>
@@ -181,6 +194,12 @@ namespace Apache.Ignite.Core.Impl.Binary
             get { return _isEnum; }
         }
 
+        /** <inheritdoc/> */
+        public IBinaryEqualityComparer EqualityComparer
+        {
+            get { return _equalityComparer; }
+        }
+
         /** <inheritDoc /> */
         public BinaryStructure WriterTypeStructure
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
index 39a2f8b..0a14bd2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObject.cs
@@ -195,6 +195,14 @@ namespace Apache.Ignite.Core.Impl.Binary
             get { return _offset; }
         }
 
+        /// <summary>
+        /// Gets the header.
+        /// </summary>
+        public BinaryObjectHeader Header
+        {
+            get { return _header; }
+        }
+
         public bool TryGetFieldPosition(string fieldName, out int pos)
         {
             var desc = _marsh.GetDescriptor(true, _header.TypeId);
@@ -244,6 +252,14 @@ namespace Apache.Ignite.Core.Impl.Binary
                 if (_data == that._data && _offset == that._offset)
                     return true;
 
+                if (TypeId != that.TypeId)
+                    return false;
+
+                var desc = _marsh.GetDescriptor(true, TypeId);
+
+                if (desc != null && desc.EqualityComparer != null)
+                    return desc.EqualityComparer.Equals(this, that);
+
                 // 1. Check headers
                 if (_header == that._header)
                 {
@@ -266,8 +282,21 @@ namespace Apache.Ignite.Core.Impl.Binary
                         object fieldVal = GetField<object>(field.Value, null);
                         object thatFieldVal = that.GetField<object>(that._fields[field.Key], null);
 
-                        if (!Equals(fieldVal, thatFieldVal))
+                        var arr = fieldVal as Array;
+                        var thatArr = thatFieldVal as Array;
+
+                        if (arr != null && thatArr != null && arr.Length == thatArr.Length)
+                        {
+                            for (var i = 0; i < arr.Length; i++)
+                            {
+                                if (!Equals(arr.GetValue(i), thatArr.GetValue(i)))
+                                    return false;
+                            }
+                        }
+                        else if (!Equals(fieldVal, thatFieldVal))
+                        {
                             return false;
+                        }
                     }
 
                     // 4. Check if objects have the same raw data.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
index 1626a2d..db18638 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectBuilder.cs
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         private IDictionary<int, BinaryBuilderField> _cache;
 
         /** Hash code. */
-        private int _hashCode;
+        private int? _hashCode;
         
         /** Current context. */
         private Context _ctx;
@@ -87,15 +87,21 @@ namespace Apache.Ignite.Core.Impl.Binary
             BinaryObject obj, IBinaryTypeDescriptor desc)
         {
             Debug.Assert(binary != null);
-            Debug.Assert(obj != null);
             Debug.Assert(desc != null);
 
             _binary = binary;
             _parent = parent ?? this;
-            _obj = obj;
             _desc = desc;
 
-            _hashCode = obj.GetHashCode();
+            if (obj != null)
+            {
+                _obj = obj;
+                _hashCode = obj.GetHashCode();
+            }
+            else
+            {
+                _obj = BinaryFromDescriptor(desc);
+            }
         }
 
         /** <inheritDoc /> */
@@ -508,7 +514,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             BinaryHeapStream inStream,
             BinaryHeapStream outStream,
             IBinaryTypeDescriptor desc,
-            int hashCode, 
+            int? hashCode, 
             IDictionary<string, BinaryBuilderField> vals)
         {
             // Set correct builder to writer frame.
@@ -578,7 +584,7 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="vals">Values to be replaced.</param>
         /// <returns>Mutated object.</returns>
         private void Mutate0(Context ctx, BinaryHeapStream inStream, IBinaryStream outStream,
-            bool changeHash, int hash, IDictionary<int, BinaryBuilderField> vals)
+            bool changeHash, int? hash, IDictionary<int, BinaryBuilderField> vals)
         {
             int inStartPos = inStream.Position;
             int outStartPos = outStream.Position;
@@ -730,7 +736,25 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                             var outLen = outStream.Position - outStartPos;
 
-                            var outHash = changeHash ? hash : inHeader.HashCode;
+                            var outHash = inHeader.HashCode;
+
+                            if (changeHash)
+                            {
+                                if (hash != null)
+                                {
+                                    outHash = hash.Value;
+                                }
+                                else
+                                {
+                                    // Get from identity resolver.
+                                    outHash = _desc.EqualityComparer != null
+                                        ? _desc.EqualityComparer.GetHashCode(outStream,
+                                            outStartPos + BinaryObjectHeader.Size,
+                                            schemaPos - outStartPos - BinaryObjectHeader.Size,
+                                            outSchema, outSchemaId, _binary.Marshaller, _desc)
+                                        : 0;
+                                }
+                            }
 
                             var outHeader = new BinaryObjectHeader(inHeader.TypeId, outHash, outLen, 
                                 outSchemaId, outSchemaOff, flags);
@@ -1027,6 +1051,30 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Create empty binary object from descriptor.
+        /// </summary>
+        /// <param name="desc">Descriptor.</param>
+        /// <returns>Empty binary object.</returns>
+        private BinaryObject BinaryFromDescriptor(IBinaryTypeDescriptor desc)
+        {
+            const int len = BinaryObjectHeader.Size;
+
+            var flags = desc.UserType ? BinaryObjectHeader.Flag.UserType : BinaryObjectHeader.Flag.None;
+
+            if (_binary.Marshaller.CompactFooter && desc.UserType)
+                flags |= BinaryObjectHeader.Flag.CompactFooter;
+
+            var hdr = new BinaryObjectHeader(desc.TypeId, 0, len, 0, len, flags);
+
+            using (var stream = new BinaryHeapStream(len))
+            {
+                BinaryObjectHeader.Write(hdr, stream, 0);
+
+                return new BinaryObject(_binary.Marshaller, stream.InternalArray, 0, hdr);
+            }
+        }
+
+        /// <summary>
         /// Mutation context.
         /// </summary>
         private class Context

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
index 0e5ad2a..636b177 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectHeader.cs
@@ -211,21 +211,40 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// Gets the raw offset of this object in specified stream.
         /// </summary>
         /// <param name="stream">The stream.</param>
-        /// <param name="position">The position.</param>
+        /// <param name="position">The binary object position in the stream.</param>
         /// <returns>Raw offset.</returns>
         public int GetRawOffset(IBinaryStream stream, int position)
         {
             Debug.Assert(stream != null);
 
+            // Either schema or raw is not present - offset is in the header.
             if (!HasRaw || !HasSchema)
                 return SchemaOffset;
 
+            // Both schema and raw data are present: raw offset is in the last 4 bytes.
             stream.Seek(position + Length - 4, SeekOrigin.Begin);
 
             return stream.ReadInt();
         }
 
         /// <summary>
+        /// Gets the footer offset where raw and non-raw data ends.
+        /// </summary>
+        /// <value>Footer offset.</value>
+        public int FooterStartOffset
+        {
+            get
+            {
+                // No schema: all we have is data. There is no offset in last 4 bytes.
+                if (!HasSchema)
+                    return Length;
+
+                // There is schema. Regardless of raw data presence, footer starts with schema.
+                return SchemaOffset;
+            }
+        }
+
+        /// <summary>
         /// Writes specified header to a stream.
         /// </summary>
         /// <param name="header">The header.</param>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
index c95746a..8ad78a4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryObjectSchemaHolder.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Impl.Binary
 {
     using System;
+    using System.Collections.Generic;
     using System.Threading;
     using Apache.Ignite.Core.Impl.Binary.IO;
     using Apache.Ignite.Core.Impl.Common;
@@ -119,5 +120,26 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             return result;
         }
+
+        /// <summary>
+        /// Gets the schema.
+        /// </summary>
+        /// <param name="schemaOffset">The schema offset.</param>
+        /// <returns>Current schema as a dictionary.</returns>
+        public Dictionary<int, int> GetFullSchema(int schemaOffset)
+        {
+            var size = _idx - schemaOffset;
+
+            var result = new Dictionary<int, int>(size);
+
+            for (int i = schemaOffset; i < _idx; i++)
+            {
+                var fld = _fields[i];
+
+                result[fld.Id] = fld.Offset;
+            }
+
+            return result;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
index b572e7c..adba577 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySurrogateTypeDescriptor.cs
@@ -130,6 +130,12 @@ namespace Apache.Ignite.Core.Impl.Binary
             get { return false; }
         }
 
+        /** <inheritdoc/> */
+        public IBinaryEqualityComparer EqualityComparer
+        {
+            get { return null; }
+        }
+
         /** <inheritDoc /> */
         public BinaryStructure WriterTypeStructure
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
index ccb2d1b..bdd7137 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinarySystemHandlers.cs
@@ -614,7 +614,9 @@ namespace Apache.Ignite.Core.Impl.Binary
          */
         private static object ReadEnumArray(BinaryReader ctx, Type type)
         {
-            return BinaryUtils.ReadTypedArray(ctx, true, type.GetElementType());
+            var elemType = type.GetElementType() ?? typeof(object);
+
+            return BinaryUtils.ReadTypedArray(ctx, true, elemType);
         }
 
         /**
@@ -622,7 +624,7 @@ namespace Apache.Ignite.Core.Impl.Binary
          */
         private static object ReadArray(BinaryReader ctx, Type type)
         {
-            var elemType = type.IsArray ? type.GetElementType() : typeof(object);
+            var elemType = type.GetElementType() ?? typeof(object);
 
             return BinaryUtils.ReadTypedArray(ctx, true, elemType);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
index 77a22dd..47b0663 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryWriter.cs
@@ -1195,9 +1195,10 @@ namespace Apache.Ignite.Core.Impl.Binary
                 {
                     // Write object fields.
                     desc.Serializer.WriteBinary(obj, this);
+                    var dataEnd = _stream.Position;
 
                     // Write schema
-                    var schemaOffset = _stream.Position - pos;
+                    var schemaOffset = dataEnd - pos;
 
                     int schemaId;
                     
@@ -1230,8 +1231,12 @@ namespace Apache.Ignite.Core.Impl.Binary
 
                     var len = _stream.Position - pos;
 
-                    var header = new BinaryObjectHeader(desc.TypeId, obj.GetHashCode(), len,
-                        schemaId, schemaOffset, flags);
+                    var hashCode = desc.EqualityComparer != null
+                        ? desc.EqualityComparer.GetHashCode(Stream, pos + BinaryObjectHeader.Size,
+                            dataEnd - pos - BinaryObjectHeader.Size, _schema, schemaIdx, _marsh, desc)
+                        : obj.GetHashCode();
+
+                    var header = new BinaryObjectHeader(desc.TypeId, hashCode, len, schemaId, schemaOffset, flags);
 
                     BinaryObjectHeader.Write(header, _stream, pos);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
index b80348e..6adb847 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/DateTimeHolder.cs
@@ -24,7 +24,7 @@ namespace Apache.Ignite.Core.Impl.Binary
     /// <summary>
     /// Wraps DateTime item in a binarizable.
     /// </summary>
-    internal class DateTimeHolder : IBinaryWriteAware
+    internal struct DateTimeHolder : IBinaryWriteAware
     {
         /** */
         private readonly DateTime _item;
@@ -64,5 +64,18 @@ namespace Apache.Ignite.Core.Impl.Binary
 
             writer.GetRawWriter().WriteLong(_item.ToBinary());
         }
+
+        /** <inheritDoc /> */
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) return false;
+            return obj is DateTimeHolder && _item.Equals(((DateTimeHolder) obj)._item);
+        }
+
+        /** <inheritDoc /> */
+        public override int GetHashCode()
+        {
+            return _item.GetHashCode();
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryEqualityComparer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryEqualityComparer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryEqualityComparer.cs
new file mode 100644
index 0000000..9628688
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryEqualityComparer.cs
@@ -0,0 +1,53 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Binary
+{
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+
+    /// <summary>
+    /// Internal comparer interface for <see cref="BinaryTypeConfiguration.EqualityComparer"/> implementations,
+    /// provides more efficient API.
+    /// </summary>
+    internal interface IBinaryEqualityComparer
+    {
+        /// <summary>
+        /// Returns a hash code for the binary object in specified stream at specified position.
+        /// </summary>
+        /// <param name="stream">Stream.</param>
+        /// <param name="startPos">Data start position (right after the header).</param>
+        /// <param name="length">Data length (without header and schema).</param>
+        /// <param name="schema">Schema holder.</param>
+        /// <param name="schemaId">Schema identifier.</param>
+        /// <param name="marshaller">Marshaller.</param>
+        /// <param name="desc">Type descriptor.</param>
+        /// <returns>
+        /// A hash code for the object in the stream.
+        /// </returns>
+        int GetHashCode(IBinaryStream stream, int startPos, int length, BinaryObjectSchemaHolder schema, int schemaId,
+            Marshaller marshaller, IBinaryTypeDescriptor desc);
+
+        /// <summary>
+        /// Returns a value indicating that two binary object are equal.
+        /// </summary>
+        /// <param name="x">First object.</param>
+        /// <param name="y">Second object.</param>
+        /// <returns>True when objects are equal; otherwise false.</returns>
+        bool Equals(IBinaryObject x, IBinaryObject y);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryTypeDescriptor.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryTypeDescriptor.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryTypeDescriptor.cs
index e25d720..6c7e360 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryTypeDescriptor.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/IBinaryTypeDescriptor.cs
@@ -78,6 +78,11 @@ namespace Apache.Ignite.Core.Impl.Binary
         bool IsEnum { get; }
 
         /// <summary>
+        /// Gets the equality comparer.
+        /// </summary>
+        IBinaryEqualityComparer EqualityComparer { get; }
+
+        /// <summary>
         /// Write type structure.
         /// </summary>
         BinaryStructure WriterTypeStructure { get; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryHeapStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryHeapStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryHeapStream.cs
index 2ce2138..ad35ff9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryHeapStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryHeapStream.cs
@@ -420,6 +420,15 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         }
 
         /** <inheritdoc /> */
+        public override T Apply<TArg, T>(IBinaryStreamProcessor<TArg, T> proc, TArg arg)
+        {
+            fixed (byte* data0 = _data)
+            {
+                return proc.Invoke(data0, arg);
+            }
+        }
+
+        /** <inheritdoc /> */
         protected override void Dispose(bool disposing)
         {
             // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
index 184209f..0b855f8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/BinaryStreamBase.cs
@@ -1119,6 +1119,19 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
             return Pos;
         }
 
+        /// <summary>
+        /// Returns a hash code for the specified byte range.
+        /// </summary>
+        public abstract T Apply<TArg, T>(IBinaryStreamProcessor<TArg, T> proc, TArg arg);
+
+        /// <summary>
+        /// Flushes the data to underlying storage.
+        /// </summary>
+        public void Flush()
+        {
+            // No-op.
+        }
+
         /** <inheritdoc /> */
         public void Dispose()
         {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
index cd509c6..3a46515 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Io/IBinaryStream.cs
@@ -318,5 +318,30 @@ namespace Apache.Ignite.Core.Impl.Binary.IO
         /// <param name="origin">Seek origin.</param>
         /// <returns>Position.</returns>
         int Seek(int offset, SeekOrigin origin);
+
+        /// <summary>
+        /// Applies specified processor to the raw stream data.
+        /// </summary>
+        T Apply<TArg, T>(IBinaryStreamProcessor<TArg, T> proc, TArg arg);
+
+        /// <summary>
+        /// Flushes the data to underlying storage.
+        /// </summary>
+        void Flush();
+    }
+
+    /// <summary>
+    /// Binary stream processor.
+    /// </summary>
+    [CLSCompliant(false)]
+    public unsafe interface IBinaryStreamProcessor<in TArg, out T>
+    {
+        /// <summary>
+        /// Invokes the processor.
+        /// </summary>
+        /// <param name="data">Data.</param>
+        /// <param name="arg">Argument.</param>
+        /// <returns>Result.</returns>
+        T Invoke(byte* data, TArg arg);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
index 475762a..6dee998 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs
@@ -408,7 +408,7 @@ namespace Apache.Ignite.Core.Impl.Binary
             if (meta != BinaryType.Empty)
             {
                 desc = new BinaryFullTypeDescriptor(null, meta.TypeId, meta.TypeName, true, null, null, null, false, 
-                    meta.AffinityKeyFieldName, meta.IsEnum);
+                    meta.AffinityKeyFieldName, meta.IsEnum, null);
 
                 _idToDesc.GetOrAdd(typeKey, _ => desc);
 
@@ -419,6 +419,14 @@ namespace Apache.Ignite.Core.Impl.Binary
         }
 
         /// <summary>
+        /// Gets the user type descriptors.
+        /// </summary>
+        public ICollection<IBinaryTypeDescriptor> GetUserTypeDescriptors()
+        {
+            return _typeNameToDesc.Values;
+        }
+
+        /// <summary>
         /// Add user type.
         /// </summary>
         /// <param name="cfg">Configuration.</param>
@@ -453,7 +461,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 var serializer = GetSerializer(cfg, typeCfg, type, typeId, nameMapper, idMapper);
 
                 AddType(type, typeId, typeName, true, keepDeserialized, nameMapper, idMapper, serializer,
-                    affKeyFld, type.IsEnum);
+                    affKeyFld, type.IsEnum, typeCfg.EqualityComparer);
             }
             else
             {
@@ -463,7 +471,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 int typeId = BinaryUtils.TypeId(typeName, nameMapper, idMapper);
 
                 AddType(null, typeId, typeName, true, keepDeserialized, nameMapper, idMapper, null,
-                    typeCfg.AffinityKeyFieldName, typeCfg.IsEnum);
+                    typeCfg.AffinityKeyFieldName, typeCfg.IsEnum, typeCfg.EqualityComparer);
             }
         }
 
@@ -521,9 +529,11 @@ namespace Apache.Ignite.Core.Impl.Binary
         /// <param name="serializer">Serializer.</param>
         /// <param name="affKeyFieldName">Affinity key field name.</param>
         /// <param name="isEnum">Enum flag.</param>
+        /// <param name="comparer">Comparer.</param>
         private void AddType(Type type, int typeId, string typeName, bool userType, 
             bool keepDeserialized, IBinaryNameMapper nameMapper, IBinaryIdMapper idMapper,
-            IBinarySerializerInternal serializer, string affKeyFieldName, bool isEnum)
+            IBinarySerializerInternal serializer, string affKeyFieldName, bool isEnum, 
+            IEqualityComparer<IBinaryObject> comparer)
         {
             long typeKey = BinaryUtils.TypeKey(userType, typeId);
 
@@ -545,7 +555,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 throw new BinaryObjectException("Conflicting type name: " + typeName);
 
             var descriptor = new BinaryFullTypeDescriptor(type, typeId, typeName, userType, nameMapper, idMapper, 
-                serializer, keepDeserialized, affKeyFieldName, isEnum);
+                serializer, keepDeserialized, affKeyFieldName, isEnum, comparer);
 
             if (type != null)
                 _typeToDesc[type] = descriptor;
@@ -571,7 +581,7 @@ namespace Apache.Ignite.Core.Impl.Binary
                 typeId = BinaryUtils.TypeId(type.Name, null, null);
 
             AddType(type, typeId, BinaryUtils.GetTypeName(type), false, false, null, null, serializer, affKeyFldName,
-                false);
+                false, null);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableObjectHolder.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableObjectHolder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableObjectHolder.cs
index 99c8f49..26b1d5f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableObjectHolder.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/SerializableObjectHolder.cs
@@ -76,5 +76,21 @@ namespace Apache.Ignite.Core.Impl.Binary
                 _item = new BinaryFormatter().Deserialize(streamAdapter, null);
             }
         }
+
+        /** <inheritdoc /> */
+        public override bool Equals(object obj)
+        {
+            if (ReferenceEquals(null, obj)) return false;
+            if (ReferenceEquals(this, obj)) return true;
+            if (obj.GetType() != GetType()) return false;
+
+            return Equals(_item, ((SerializableObjectHolder) obj)._item);
+        }
+
+        /** <inheritdoc /> */
+        public override int GetHashCode()
+        {
+            return _item != null ? _item.GetHashCode() : 0;
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
index e1df50b..bb9cbfc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs
@@ -142,7 +142,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /// </summary>
         private static void WriteComplexProperty(object obj, XmlWriter writer, Type valueType)
         {
-            var props = GetNonDefaultProperties(obj).ToList();
+            var props = GetNonDefaultProperties(obj).OrderBy(x => x.Name).ToList();
 
             // Specify type for interfaces and abstract classes
             if (valueType.IsAbstract)
@@ -353,7 +353,8 @@ namespace Apache.Ignite.Core.Impl.Common
         /// </summary>
         private static List<Type> GetConcreteDerivedTypes(Type type)
         {
-            return type.Assembly.GetTypes().Where(t => t.IsClass && !t.IsAbstract && type.IsAssignableFrom(t)).ToList();
+            return typeof(IIgnite).Assembly.GetTypes()
+                .Where(t => t.IsClass && !t.IsAbstract && type.IsAssignableFrom(t)).ToList();
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
index c758d28..3719846 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformMemoryStream.cs
@@ -732,6 +732,22 @@ namespace Apache.Ignite.Core.Impl.Memory
         }
 
         /// <summary>
+        /// Returns a hash code for the specified byte range.
+        /// </summary>
+        public T Apply<TArg, T>(IBinaryStreamProcessor<TArg, T> proc, TArg arg)
+        {
+            return proc.Invoke(_data, arg);
+        }
+
+        /// <summary>
+        /// Flushes the data to underlying storage.
+        /// </summary>
+        public void Flush()
+        {
+            SynchronizeOutput();
+        }
+
+        /// <summary>
         /// Ensure capacity for write and shift position.
         /// </summary>
         /// <param name="cnt">Bytes count.</param>


[27/50] [abbrv] ignite git commit: IGNITE-4518 Fixed parallel load of cache. - Fixes #1426.

Posted by sb...@apache.org.
IGNITE-4518 Fixed parallel load of cache. - Fixes #1426.

Signed-off-by: Andrey Novikov <an...@gridgain.com>


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 79401b2ebf814eeffa36d41a82d1238d8eccc7e9
Parents: 6045a24
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Jan 16 10:33:16 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Jan 16 10:33:16 2017 +0700

----------------------------------------------------------------------
 .../store/jdbc/dialect/BasicJdbcDialect.java    | 31 ++++++++-----
 .../store/jdbc/CacheJdbcPojoStoreTest.java      | 48 +++++++++++++++++++-
 2 files changed, 66 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/79401b2e/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
index 3ab112a..139f3fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/dialect/BasicJdbcDialect.java
@@ -173,13 +173,15 @@ public class BasicJdbcDialect implements JdbcDialect {
         if (appendLowerBound) {
             sb.a("(");
 
-            for (int cnt = keyCols.size(); cnt > 0; cnt--) {
-                for (int j = 0; j < cnt; j++)
-                    if (j == cnt - 1)
-                        sb.a(cols[j]).a(" > ? ");
+            for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) {
+                for (int idx = 0; idx < keyCnt; idx++) {
+                    if (idx == keyCnt - 1)
+                        sb.a(cols[idx]).a(" > ? ");
                     else
-                        sb.a(cols[j]).a(" = ? AND ");
-                if (cnt != 1)
+                        sb.a(cols[idx]).a(" = ? AND ");
+                }
+
+                if (keyCnt != 1)
                     sb.a("OR ");
             }
 
@@ -192,13 +194,18 @@ public class BasicJdbcDialect implements JdbcDialect {
         if (appendUpperBound) {
             sb.a("(");
 
-            for (int cnt = keyCols.size(); cnt > 0; cnt--) {
-                for (int j = 0; j < cnt; j++)
-                    if (j == cnt - 1)
-                        sb.a(cols[j]).a(" <= ? ");
+            for (int keyCnt = keyCols.size(); keyCnt > 0; keyCnt--) {
+                for (int idx = 0, lastIdx = keyCnt - 1; idx < keyCnt; idx++) {
+                    sb.a(cols[idx]);
+
+                    // For composite key when not all of the key columns are constrained should use < (strictly less).
+                    if (idx == lastIdx)
+                        sb.a(keyCnt == keyCols.size() ? " <= ? " : " < ? ");
                     else
-                        sb.a(cols[j]).a(" = ? AND ");
-                if (cnt != 1)
+                        sb.a(" = ? AND ");
+                }
+
+                if (keyCnt != 1)
                     sb.a(" OR ");
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/79401b2e/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
index d8f75d3..4a0b1da 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcPojoStoreTest.java
@@ -216,7 +216,7 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
 
         stmt.executeUpdate("CREATE TABLE IF NOT EXISTS " +
             "Person_Complex (id integer not null, org_id integer not null, city_id integer not null, " +
-            "name varchar(50), salary integer, PRIMARY KEY(id))");
+            "name varchar(50), salary integer, PRIMARY KEY(id, org_id, city_id))");
 
         conn.commit();
 
@@ -352,6 +352,52 @@ public class CacheJdbcPojoStoreTest extends GridAbstractCacheStoreSelfTest<Cache
     /**
      * @throws Exception If failed.
      */
+    public void testParallelLoad() throws Exception {
+        Connection conn = store.openConnection(false);
+
+        PreparedStatement prnComplexStmt = conn.prepareStatement("INSERT INTO Person_Complex(id, org_id, city_id, name, salary) VALUES (?, ?, ?, ?, ?)");
+
+        for (int i = 0; i < 8; i++) {
+
+            prnComplexStmt.setInt(1, (i >> 2) & 1);
+            prnComplexStmt.setInt(2, (i >> 1) & 1);
+            prnComplexStmt.setInt(3, i % 2);
+
+            prnComplexStmt.setString(4, "name");
+            prnComplexStmt.setInt(5, 1000 + i * 500);
+
+            prnComplexStmt.addBatch();
+        }
+
+        prnComplexStmt.executeBatch();
+
+        U.closeQuiet(prnComplexStmt);
+
+        conn.commit();
+
+        U.closeQuiet(conn);
+
+        final Collection<PersonComplexKey> prnComplexKeys = new ConcurrentLinkedQueue<>();
+
+        IgniteBiInClosure<Object, Object> c = new CI2<Object, Object>() {
+            @Override public void apply(Object k, Object v) {
+                if (k instanceof PersonComplexKey && v instanceof Person)
+                    prnComplexKeys.add((PersonComplexKey)k);
+                else
+                    fail("Unexpected entry [key=" + k + ", value=" + v + "]");
+            }
+        };
+
+        store.setParallelLoadCacheMinimumThreshold(2);
+
+        store.loadCache(c);
+
+        assertEquals(8, prnComplexKeys.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testWriteRetry() throws Exception {
         CacheJdbcPojoStore<Object, Object> store = store();
 


[22/50] [abbrv] ignite git commit: updated 'setCollocated' flag documentation

Posted by sb...@apache.org.
updated 'setCollocated' flag documentation


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 1f358db1eb3c9a28dd1a66042539796c1ef5ac45
Parents: ff0caf8
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jan 11 13:43:39 2017 -0800
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jan 11 13:43:39 2017 -0800

----------------------------------------------------------------------
 .../java/org/apache/ignite/cache/query/SqlFieldsQuery.java     | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f358db1/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
index 9b17e78..eac8cfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/SqlFieldsQuery.java
@@ -164,6 +164,12 @@ public class SqlFieldsQuery extends Query<List<?>> {
     /**
      * Sets flag defining if this query is collocated.
      *
+     * Collocation flag is used for optimization purposes of queries with GROUP BY statements.
+     * Whenever Ignite executes a distributed query, it sends sub-queries to individual cluster members.
+     * If you know in advance that the elements of your query selection are collocated together on the same node and
+     * you group by collocated key (primary or affinity key), then Ignite can make significant performance and network
+     * optimizations by grouping data on remote nodes.
+     *
      * @param collocated Flag value.
      * @return {@code this} For chaining.
      */


[30/50] [abbrv] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by sb...@apache.org.
Merge branch 'master' into ignite-2.0


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 77ca2e636c73e464f833f227c4894df0785ae9e2
Parents: 6090ebd 82dd912
Author: devozerov <vo...@gridgain.com>
Authored: Mon Jan 16 16:07:49 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jan 16 16:07:49 2017 +0300

----------------------------------------------------------------------
 .../ignite/cache/query/SqlFieldsQuery.java      |   6 +
 .../store/jdbc/dialect/BasicJdbcDialect.java    |  31 +-
 .../GridClientConnectionManagerAdapter.java     |   7 +-
 .../impl/connection/GridClientTopology.java     |  53 +++-
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 ++-
 .../GridNearAtomicSingleUpdateFuture.java       |  44 +--
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 ++--
 .../IgniteCacheObjectProcessorImpl.java         |   5 +-
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |  48 +++-
 .../ignite/cache/store/jdbc/model/Person.java   |   2 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   6 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   2 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   5 +-
 .../AtomicPutAllChangingTopologyTest.java       | 212 ++++++++++++++
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../apache/ignite/stream/flume/IgniteSink.java  |   7 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  64 +++--
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   6 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  18 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   4 +-
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |   1 +
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++++++-
 modules/platforms/cpp/odbc/Makefile.am          |   3 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++++++---
 .../cpp/odbc/include/ignite/odbc/utility.h      |  19 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 .../cpp/odbc/src/config/connection_info.cpp     |   3 -
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 +++
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 ++--
 modules/platforms/cpp/odbc/src/log.cpp          |  83 ++++++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 283 ++++++++-----------
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 ++++++++++--
 modules/platforms/cpp/odbc/src/utility.cpp      |  18 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |   7 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.AspNet.csproj                 |   5 +
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Benchmarks.csproj             |   5 +
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   5 +
 .../Apache.Ignite.Core.Tests.csproj             |   6 +
 .../Cache/CacheAbstractTest.cs                  |   2 +
 .../Log/ConcurrentMemoryTarget.cs               |  73 +++++
 .../Log/NLogLoggerTest.cs                       |   5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   6 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   1 -
 .../Apache.Ignite.Core.csproj                   |  16 +-
 .../Impl/Binary/BinaryReader.cs                 |  16 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   4 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |   5 +
 .../Apache.Ignite.EntityFramework.csproj        |   9 +-
 .../Impl/ArrayDbDataReader.cs                   |   5 +
 .../Impl/DataReaderResult.cs                    |   2 +
 .../Impl/DbCommandInfo.cs                       |  11 +
 .../Impl/DbCommandProxy.cs                      |   1 +
 .../Apache.Ignite.Linq.csproj                   |   5 +
 .../Apache.Ignite.Log4Net.csproj                |   5 +
 .../IgniteLog4NetLogger.cs                      |  16 +-
 .../Apache.Ignite.NLog.csproj                   |   5 +
 modules/platforms/dotnet/Apache.Ignite.sln      |   3 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   5 +
 modules/platforms/dotnet/README.md              |   6 +-
 modules/platforms/dotnet/build.ps1              |   2 +-
 .../Apache.Ignite.Examples.csproj               |   3 +
 .../Apache.Ignite.ExamplesDll.csproj            |   4 +
 .../Services/MapService.cs                      |   2 -
 86 files changed, 1602 insertions(+), 528 deletions(-)
----------------------------------------------------------------------



[42/50] [abbrv] ignite git commit: Merge master into ignite-3477

Posted by sb...@apache.org.
Merge master into ignite-3477


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

Branch: refs/heads/ignite-gg-11810-1
Commit: cb60e3865acb738f7f23124ba4f3e41c0dd63cfa
Parents: 4538818
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 17 13:48:56 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 17 13:48:56 2017 +0300

----------------------------------------------------------------------
 .../processors/platform/utils/PlatformConfigurationUtils.java | 7 +++----
 1 file changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cb60e386/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 33571ad..5fcd2eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -21,12 +21,12 @@ import java.lang.management.ManagementFactory;
 import java.net.InetSocketAddress;
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Set;
-import java.util.HashSet;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import javax.cache.configuration.Factory;
 import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.binary.BinaryArrayIdentityResolver;
@@ -58,13 +58,12 @@ import org.apache.ignite.configuration.TransactionConfiguration;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction;
+import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory;
 import org.apache.ignite.platform.dotnet.PlatformDotNetAffinityFunction;
 import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryConfiguration;
 import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryTypeConfiguration;
 import org.apache.ignite.platform.dotnet.PlatformDotNetCacheStoreFactoryNative;
 import org.apache.ignite.platform.dotnet.PlatformDotNetConfiguration;
-import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory;
-import org.apache.ignite.platform.dotnet.*;
 import org.apache.ignite.spi.communication.CommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpiMBean;


[02/50] [abbrv] ignite git commit: IGNITE-4458: Hadoop: "striped" shuffle mode is default from now on. This closes #1390.

Posted by sb...@apache.org.
IGNITE-4458: Hadoop: "striped" shuffle mode is default from now on. This closes #1390.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 1f743465d6875ef48b1835d03a78a0dbaf339bf6
Parents: a9b1fc2
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu Dec 29 11:14:10 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 29 11:14:10 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/hadoop/HadoopJobProperty.java       | 2 +-
 .../internal/processors/hadoop/shuffle/HadoopShuffleJob.java       | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f743465/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 4398acd..a3115bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -105,7 +105,7 @@ public enum HadoopJobProperty {
     /**
      * Whether to stripe mapper output for remote reducers.
      * <p>
-     * Defaults to {@code false}.
+     * Defaults to {@code true}.
      */
     SHUFFLE_MAPPER_STRIPED_OUTPUT("ignite.shuffle.mapper.striped.output"),
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f743465/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 1c546a1..7713d6d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -169,7 +169,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
         this.embedded = embedded;
 
         // No stripes for combiner.
-        boolean stripeMappers0 = get(job.info(), SHUFFLE_MAPPER_STRIPED_OUTPUT, false);
+        boolean stripeMappers0 = get(job.info(), SHUFFLE_MAPPER_STRIPED_OUTPUT, true);
 
         if (stripeMappers0) {
             if (job.info().hasCombiner()) {


[11/50] [abbrv] ignite git commit: Merge branch 'ignite-1.7.5'

Posted by sb...@apache.org.
Merge branch 'ignite-1.7.5'

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
#	modules/core/src/main/resources/META-INF/classnames.properties
#	modules/platforms/cpp/configure.ac
#	modules/platforms/cpp/configure.acrel
#	modules/platforms/cpp/examples/configure.ac
#	modules/platforms/cpp/odbc/install/ignite-odbc-amd64.wxs
#	modules/platforms/cpp/odbc/install/ignite-odbc-x86.wxs
#	modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Benchmarks/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests.NuGet/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
#	modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Core/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Linq/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.Log4Net/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite.NLog/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/Apache.Ignite/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/examples/Apache.Ignite.Examples/Properties/AssemblyInfo.cs
#	modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Properties/AssemblyInfo.cs
#	modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js


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

Branch: refs/heads/ignite-gg-11810-1
Commit: beb242bc22aa7ca6c068fd59a09965fa5967177e
Parents: d62542b 6c38eb2
Author: devozerov <vo...@gridgain.com>
Authored: Thu Jan 5 11:44:55 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 5 11:44:55 2017 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/IgniteLogger.java    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../ignite/cache/affinity/AffinityKey.java      |   4 +-
 .../org/apache/ignite/events/CacheEvent.java    |   6 +-
 .../ignite/events/CacheQueryReadEvent.java      |   8 +-
 .../ignite/internal/binary/BinaryContext.java   |   4 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |  10 +-
 .../ignite/internal/binary/BinaryMetadata.java  |   5 +-
 .../internal/binary/BinaryObjectExImpl.java     |   8 +-
 .../ignite/internal/binary/BinaryTypeProxy.java |  15 +-
 .../ignite/internal/binary/BinaryUtils.java     |   4 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../processors/cache/CacheInvokeResult.java     |   2 +-
 .../processors/cache/CacheLazyEntry.java        |   4 +-
 .../processors/cache/CacheObjectAdapter.java    |   7 +-
 .../processors/cache/GridCacheAdapter.java      |   5 +-
 .../cache/GridCacheMvccCandidate.java           |   9 +-
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   2 +-
 .../distributed/near/GridNearLockFuture.java    |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |   4 +-
 .../cache/query/GridCacheQueryManager.java      |  13 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +
 .../cache/query/GridCacheSqlQuery.java          |   4 +-
 .../continuous/CacheContinuousQueryEvent.java   |   8 +-
 .../continuous/CacheContinuousQueryManager.java |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  30 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../GridCacheVersionConflictContext.java        |   2 +-
 .../closure/GridClosureProcessor.java           |   4 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastructures/CollocatedSetItemKey.java    |   2 +-
 .../GridCacheAtomicLongValue.java               |   2 +
 .../GridCacheAtomicSequenceImpl.java            |   2 +
 .../GridCacheAtomicSequenceValue.java           |   2 +
 .../GridCacheCountDownLatchValue.java           |   3 +
 .../datastructures/GridCacheSetItemKey.java     |   2 +-
 .../internal/processors/job/GridJobWorker.java  |   7 +-
 .../odbc/OdbcQueryExecuteRequest.java           |   6 +-
 .../platform/PlatformNativeException.java       |   3 +-
 .../processors/query/GridQueryProcessor.java    |  35 +-
 .../processors/rest/GridRestResponse.java       |   2 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   2 +-
 .../util/tostring/GridToStringBuilder.java      | 656 +++++++++++++++++--
 .../util/tostring/GridToStringInclude.java      |  12 +-
 .../util/tostring/GridToStringThreadLocal.java  |  12 +-
 .../query/VisorQueryScanSubstringFilter.java    |   5 +-
 .../internal/visor/query/VisorQueryUtils.java   |  60 ++
 .../apache/ignite/spi/indexing/IndexingSpi.java |   3 +
 .../resources/META-INF/classnames.properties    |   1 +
 .../internal/binary/BinaryEnumsSelfTest.java    |  18 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   | 199 +++++-
 .../tostring/GridToStringBuilderSelfTest.java   |  33 +-
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   9 +-
 .../org/apache/ignite/logger/jcl/JclLogger.java |   9 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |  11 +-
 64 files changed, 1145 insertions(+), 174 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 4d85c54,0da0f49..083bb72
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@@ -500,17 -502,17 +505,25 @@@ public final class IgniteSystemProperti
       * <p>
       * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture.
       */
 -    public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS";
 +    public static final String IGNITE_MEMORY_UNALIGNED_ACCESS = "IGNITE_MEMORY_UNALIGNED_ACCESS";
 +
 +    /**
 +     * When unsafe memory copy if performed below this threshold, Ignite will do it on per-byte basis instead of
 +     * calling to Unsafe.copyMemory().
 +     * <p>
 +     * Defaults to 0, meaning that threshold is disabled.
 +     */
 +    public static final String IGNITE_MEMORY_PER_BYTE_COPY_THRESHOLD = "IGNITE_MEMORY_PER_BYTE_COPY_THRESHOLD";
  
      /**
+      * When set to {@code true} BinaryObject will be unwrapped before passing to IndexingSpi to preserve
+      * old behavior query processor with IndexingSpi.
+      * <p>
+      * @deprecated Should be removed in Apache Ignite 2.0.
+      */
+     public static final String IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI = "IGNITE_UNWRAP_BINARY_FOR_INDEXING_SPI";
+ 
+     /**
       * Enforces singleton.
       */
      private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
index b80f573,5f1e3e9..5b5aeba
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
@@@ -27,9 -26,10 +27,10 @@@ import org.apache.ignite.binary.BinaryA
  import org.apache.ignite.binary.BinaryObject;
  import org.apache.ignite.binary.BinaryObjectBuilder;
  import org.apache.ignite.binary.BinaryObjectException;
 +import org.apache.ignite.binary.BinaryIdentityResolver;
  import org.apache.ignite.binary.BinaryType;
  import org.apache.ignite.internal.binary.builder.BinaryObjectBuilderImpl;
 -import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
+ import org.apache.ignite.internal.util.typedef.internal.S;
  import org.apache.ignite.internal.util.typedef.internal.SB;
  import org.apache.ignite.lang.IgniteUuid;
  import org.jetbrains.annotations.Nullable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 1594cee,6c093ee..58f94f4
--- 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
@@@ -44,9 -44,9 +44,10 @@@ import javax.cache.Cache
  import javax.cache.CacheException;
  import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.IgniteException;
+ import org.apache.ignite.IgniteSystemProperties;
  import org.apache.ignite.binary.BinaryField;
  import org.apache.ignite.binary.BinaryObject;
 +import org.apache.ignite.binary.BinaryObjectBuilder;
  import org.apache.ignite.binary.BinaryType;
  import org.apache.ignite.binary.Binarylizable;
  import org.apache.ignite.cache.CacheTypeMetadata;

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --cc modules/core/src/main/resources/META-INF/classnames.properties
index 4d0b931,8a6dc66..212e94a
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@@ -724,11 -719,9 +724,12 @@@ org.apache.ignite.internal.processors.c
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1
 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2
 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3
+ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3$1
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1
 +org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$1
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2
  org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
index 2a177ff,bf72782..c9f50d1
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/binary/GridCacheBinaryObjectsAbstractSelfTest.java
@@@ -65,10 -58,9 +65,11 @@@ import org.apache.ignite.binary.BinaryF
  import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
  import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
  import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 +import org.apache.ignite.internal.processors.cache.MapCacheStoreStrategy;
 +import org.apache.ignite.internal.util.typedef.F;
  import org.apache.ignite.internal.util.typedef.P2;
  import org.apache.ignite.internal.util.typedef.internal.CU;
+ import org.apache.ignite.internal.util.typedef.internal.S;
  import org.apache.ignite.internal.util.typedef.internal.U;
  import org.apache.ignite.lang.IgniteBiInClosure;
  import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
index 1bca0e8,d6cb3df..f5fa618
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--\ufeff/*
++\ufeff\ufeff/*
  * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
index 0926a46,9bd5c47..d72c9db
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
@@@ -1,4 -1,4 +1,4 @@@
--\ufeff/*
++\ufeff\ufeff/*
  * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/beb242bc/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --cc modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index 1fc6c59,7bf322f..cc833ea
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@@ -1,19 -1,19 +1,19 @@@
- \ufeff/*
 -\ufeff\ufeff\ufeff\ufeff/*
 - * 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.
 - */
++\ufeff\ufeff/*
 +* 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.
 +*/
  
  using System.Reflection;
  using System.Runtime.InteropServices;


[08/50] [abbrv] ignite git commit: IGNITE-4519 Updating ignite-aws java sdk pom version

Posted by sb...@apache.org.
IGNITE-4519 Updating ignite-aws java sdk pom version


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 6c1cd162e60af37c9d380d801a6d2ce7f3b448b1
Parents: 72f03ea
Author: chandresh.pancholi <ch...@arvindinternet.com>
Authored: Wed Jan 4 18:21:41 2017 +0530
Committer: chandresh.pancholi <ch...@arvindinternet.com>
Committed: Wed Jan 4 18:21:41 2017 +0530

----------------------------------------------------------------------
 parent/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c1cd162/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 682efa2..2cb88b0 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -53,7 +53,7 @@
         <aspectj.bundle.version>1.7.2_1</aspectj.bundle.version>
         <aspectj.version>1.7.2</aspectj.version>
         <aws.sdk.bundle.version>1.10.12_1</aws.sdk.bundle.version>
-        <aws.sdk.version>1.10.29</aws.sdk.version>
+        <aws.sdk.version>1.11.75</aws.sdk.version>
         <camel.version>2.16.0</camel.version>
         <commons.beanutils.bundle.version>1.8.3_1</commons.beanutils.bundle.version>
         <commons.beanutils.version>1.8.3</commons.beanutils.version>


[32/50] [abbrv] ignite git commit: IGNITE-4428: Hadoop: moved HadoopMapReducePlanner and dependent classes to public space. This closes #1389. This closes #1394.

Posted by sb...@apache.org.
IGNITE-4428: Hadoop: moved HadoopMapReducePlanner and dependent classes to public space. This closes #1389. This closes #1394.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: d14e0727b3dd61ab5ec2957133d77dbc25e9ba68
Parents: 77ca2e6
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Mon Jan 16 16:36:25 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jan 16 16:36:25 2017 +0300

----------------------------------------------------------------------
 .../configuration/HadoopConfiguration.java      |   2 +-
 .../apache/ignite/hadoop/HadoopInputSplit.java  |  54 +++++++
 .../org/apache/ignite/hadoop/HadoopJob.java     |  74 ++++++++++
 .../ignite/hadoop/HadoopMapReducePlan.java      |  80 +++++++++++
 .../ignite/hadoop/HadoopMapReducePlanner.java   |  40 ++++++
 .../processors/hadoop/HadoopDefaultJobInfo.java |   4 +-
 .../processors/hadoop/HadoopFileBlock.java      |   1 +
 .../processors/hadoop/HadoopInputSplit.java     |  54 -------
 .../internal/processors/hadoop/HadoopJob.java   | 107 --------------
 .../internal/processors/hadoop/HadoopJobEx.java | 140 +++++++++++++++++++
 .../processors/hadoop/HadoopJobInfo.java        |  54 +++----
 .../processors/hadoop/HadoopMapReducePlan.java  |  80 -----------
 .../hadoop/HadoopMapReducePlanner.java          |  40 ------
 .../processors/hadoop/HadoopTaskContext.java    |   6 +-
 .../processors/hadoop/HadoopTaskInfo.java       |   1 +
 .../hadoop/counter/HadoopCounterWriter.java     |   4 +-
 .../resources/META-INF/classnames.properties    |   4 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   6 +-
 .../IgniteHadoopWeightedMapReducePlanner.java   |  10 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 118 ++++++++++++++++
 .../planner/HadoopTestRoundRobinMrPlanner.java  |  75 ++++++++++
 .../processors/hadoop/HadoopCommonUtils.java    |   1 +
 .../processors/hadoop/HadoopContext.java        |   2 +
 .../processors/hadoop/HadoopExternalSplit.java  |   1 +
 .../processors/hadoop/HadoopSplitWrapper.java   |   1 +
 .../HadoopFileSystemCounterWriterDelegate.java  |   4 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |   4 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |   6 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |   4 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   2 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |  15 +-
 .../hadoop/impl/v2/HadoopV2Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   6 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |   4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  32 ++---
 .../planner/HadoopAbstractMapReducePlanner.java | 116 ---------------
 .../planner/HadoopDefaultMapReducePlan.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   4 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |   7 +-
 .../HadoopEmbeddedTaskExecutor.java             |   8 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   6 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   8 +-
 .../external/HadoopExternalTaskExecutor.java    |  16 +--
 .../child/HadoopChildProcessRunner.java         |   4 +-
 .../resources/META-INF/classnames.properties    |   4 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |   4 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   1 +
 .../hadoop/impl/HadoopPlannerMockJob.java       |  10 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |  16 +--
 .../hadoop/impl/HadoopTasksV1Test.java          |   4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |   4 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |  75 ----------
 .../hadoop/impl/HadoopTestTaskContext.java      |   6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |   6 +-
 .../HadoopWeightedMapReducePlannerTest.java     |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   4 +-
 57 files changed, 738 insertions(+), 615 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
index 84014d6..6443a67 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/HadoopConfiguration.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.configuration;
 
 import org.apache.ignite.lifecycle.LifecycleBean;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
+import org.apache.ignite.hadoop.HadoopMapReducePlanner;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopInputSplit.java b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopInputSplit.java
new file mode 100644
index 0000000..4138e64
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopInputSplit.java
@@ -0,0 +1,54 @@
+/*
+ * 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.hadoop;
+
+import java.io.Externalizable;
+
+/**
+ * Abstract fragment of an input data source.
+ */
+public abstract class HadoopInputSplit implements Externalizable {
+    /** */
+    protected String[] hosts;
+
+    /**
+     * Array of hosts where this input split resides.
+     *
+     * @return Hosts.
+     */
+    public String[] hosts() {
+        assert hosts != null;
+
+        return hosts;
+    }
+
+    /**
+     * This method must be implemented for purpose of internal implementation.
+     *
+     * @param obj Another object.
+     * @return {@code true} If objects are equal.
+     */
+    @Override public abstract boolean equals(Object obj);
+
+    /**
+     * This method must be implemented for purpose of internal implementation.
+     *
+     * @return Hash code of the object.
+     */
+    @Override public abstract int hashCode();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopJob.java b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopJob.java
new file mode 100644
index 0000000..8ee0330
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopJob.java
@@ -0,0 +1,74 @@
+/*
+ * 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.hadoop;
+
+import java.util.Collection;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Compact job description.
+ */
+public interface HadoopJob {
+    /**
+     * Gets collection of input splits for this job.
+     *
+     * @return Input splits.
+     */
+    public Collection<HadoopInputSplit> input();
+
+    /**
+     * Gets optional configuration property for the job.
+     *
+     * @param name Property name.
+     * @return Value or {@code null} if none.
+     */
+    @Nullable String property(String name);
+
+    /**
+     * Checks whether job has combiner.
+     *
+     * @return {@code true} If job has combiner.
+     */
+    boolean hasCombiner();
+
+    /**
+     * Checks whether job has reducer.
+     * Actual number of reducers will be in {@link HadoopMapReducePlan#reducers()}.
+     *
+     * @return Number of reducer.
+     */
+    boolean hasReducer();
+
+    /**
+     * @return Number of reducers configured for job.
+     */
+    int reducers();
+
+    /**
+     * Gets job name.
+     *
+     * @return Job name.
+     */
+    String jobName();
+
+    /**
+     * Gets user name.
+     *
+     * @return User name.
+     */
+    String user();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlan.java b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlan.java
new file mode 100644
index 0000000..f77fb64
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlan.java
@@ -0,0 +1,80 @@
+/*
+ * 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.hadoop;
+
+import java.io.Serializable;
+import java.util.Collection;
+import java.util.UUID;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Map-reduce job execution plan.
+ */
+public interface HadoopMapReducePlan extends Serializable {
+    /**
+     * Gets collection of file blocks for which mappers should be executed.
+     *
+     * @param nodeId Node ID to check.
+     * @return Collection of file blocks or {@code null} if no mappers should be executed on given node.
+     */
+    @Nullable public Collection<HadoopInputSplit> mappers(UUID nodeId);
+
+    /**
+     * Gets reducer IDs that should be started on given node.
+     *
+     * @param nodeId Node ID to check.
+     * @return Array of reducer IDs.
+     */
+    @Nullable public int[] reducers(UUID nodeId);
+
+    /**
+     * Gets collection of all node IDs involved in map part of job execution.
+     *
+     * @return Collection of node IDs.
+     */
+    public Collection<UUID> mapperNodeIds();
+
+    /**
+     * Gets collection of all node IDs involved in reduce part of job execution.
+     *
+     * @return Collection of node IDs.
+     */
+    public Collection<UUID> reducerNodeIds();
+
+    /**
+     * Gets overall number of mappers for the job.
+     *
+     * @return Number of mappers.
+     */
+    public int mappers();
+
+    /**
+     * Gets overall number of reducers for the job.
+     *
+     * @return Number of reducers.
+     */
+    public int reducers();
+
+    /**
+     * Gets node ID for reducer.
+     *
+     * @param reducer Reducer.
+     * @return Node ID.
+     */
+    public UUID nodeForReducer(int reducer);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlanner.java b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlanner.java
new file mode 100644
index 0000000..8d77b70
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/HadoopMapReducePlanner.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.hadoop;
+
+import java.util.Collection;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Map-reduce execution planner.
+ */
+public interface HadoopMapReducePlanner {
+    /**
+     * Prepares map-reduce execution plan for the given job and topology.
+     *
+     * @param job Job.
+     * @param top Topology.
+     * @param oldPlan Old plan in case of partial failure.
+     * @return Map reduce plan.
+     * @throws IgniteCheckedException If an error occurs.
+     */
+    public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
+        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index ae17ac8..ab38e4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -82,12 +82,12 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls, HadoopJobId jobId, IgniteLogger log,
+    @Override public HadoopJobEx createJob(Class<? extends HadoopJobEx> jobCls, HadoopJobId jobId, IgniteLogger log,
         @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException {
         assert jobCls != null;
 
         try {
-            Constructor<? extends HadoopJob> constructor = jobCls.getConstructor(HadoopJobId.class,
+            Constructor<? extends HadoopJobEx> constructor = jobCls.getConstructor(HadoopJobId.class,
                 HadoopDefaultJobInfo.class, IgniteLogger.class, String[].class, HadoopHelper.class);
 
             return constructor.newInstance(jobId, this, log, libNames, helper);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopFileBlock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopFileBlock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopFileBlock.java
index bc665eb..351abce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopFileBlock.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopFileBlock.java
@@ -22,6 +22,7 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.net.URI;
 import java.util.Arrays;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.S;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopInputSplit.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopInputSplit.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopInputSplit.java
deleted file mode 100644
index 998cb61..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopInputSplit.java
+++ /dev/null
@@ -1,54 +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.hadoop;
-
-import java.io.Externalizable;
-
-/**
- * Abstract fragment of an input data source.
- */
-public abstract class HadoopInputSplit implements Externalizable {
-    /** */
-    protected String[] hosts;
-
-    /**
-     * Array of hosts where this input split resides.
-     *
-     * @return Hosts.
-     */
-    public String[] hosts() {
-        assert hosts != null;
-
-        return hosts;
-    }
-
-    /**
-     * This method must be implemented for purpose of internal implementation.
-     *
-     * @param obj Another object.
-     * @return {@code true} If objects are equal.
-     */
-    @Override public abstract boolean equals(Object obj);
-
-    /**
-     * This method must be implemented for purpose of internal implementation.
-     *
-     * @return Hash code of the object.
-     */
-    @Override public abstract int hashCode();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
deleted file mode 100644
index a77c744..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
+++ /dev/null
@@ -1,107 +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.hadoop;
-
-import java.util.Collection;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-
-/**
- * Hadoop job.
- */
-public interface HadoopJob {
-    /**
-     * Gets job ID.
-     *
-     * @return Job ID.
-     */
-    public HadoopJobId id();
-
-    /**
-     * Gets job information.
-     *
-     * @return Job information.
-     */
-    public HadoopJobInfo info();
-
-    /**
-     * Gets collection of input splits for this job.
-     *
-     * @return Input splits.
-     */
-    public Collection<HadoopInputSplit> input() throws IgniteCheckedException;
-
-    /**
-     * Returns context for task execution.
-     *
-     * @param info Task info.
-     * @return Task Context.
-     * @throws IgniteCheckedException If failed.
-     */
-    public HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Does all the needed initialization for the job. Will be called on each node where tasks for this job must
-     * be executed.
-     * <p>
-     * If job is running in external mode this method will be called on instance in Ignite node with parameter
-     * {@code false} and on instance in external process with parameter {@code true}.
-     *
-     * @param external If {@code true} then this job instance resides in external process.
-     * @param locNodeId Local node ID.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException;
-
-    /**
-     * Release all the resources.
-     * <p>
-     * If job is running in external mode this method will be called on instance in Ignite node with parameter
-     * {@code false} and on instance in external process with parameter {@code true}.
-     *
-     * @param external If {@code true} then this job instance resides in external process.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void dispose(boolean external) throws IgniteCheckedException;
-
-    /**
-     * Prepare local environment for the task.
-     *
-     * @param info Task info.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void prepareTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Cleans up local environment of the task.
-     *
-     * @param info Task info.
-     * @throws IgniteCheckedException If failed.
-     */
-    public void cleanupTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException;
-
-    /**
-     * Cleans up the job staging directory.
-     */
-    public void cleanupStagingDirectory();
-
-    /**
-     * @return Ignite work directory.
-     */
-    public String igniteWorkDirectory();
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobEx.java
new file mode 100644
index 0000000..ba78af9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobEx.java
@@ -0,0 +1,140 @@
+/*
+ * 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.hadoop;
+
+import java.util.Collection;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopJob;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Hadoop job.
+ */
+public abstract class HadoopJobEx implements HadoopJob {
+    /**
+     * Gets job ID.
+     *
+     * @return Job ID.
+     */
+    abstract public HadoopJobId id();
+
+    /**
+     * Gets job information.
+     *
+     * @return Job information.
+     */
+    abstract public HadoopJobInfo info();
+
+    /**
+     * Gets collection of input splits for this job.
+     *
+     * @return Input splits.
+     */
+    abstract public Collection<HadoopInputSplit> input();
+
+    /**
+     * Returns context for task execution.
+     *
+     * @param info Task info.
+     * @return Task Context.
+     * @throws IgniteCheckedException If failed.
+     */
+    abstract public HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException;
+
+    /**
+     * Does all the needed initialization for the job. Will be called on each node where tasks for this job must
+     * be executed.
+     * <p>
+     * If job is running in external mode this method will be called on instance in Ignite node with parameter
+     * {@code false} and on instance in external process with parameter {@code true}.
+     *
+     * @param external If {@code true} then this job instance resides in external process.
+     * @param locNodeId Local node ID.
+     * @throws IgniteCheckedException If failed.
+     */
+    abstract public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException;
+
+    /**
+     * Release all the resources.
+     * <p>
+     * If job is running in external mode this method will be called on instance in Ignite node with parameter
+     * {@code false} and on instance in external process with parameter {@code true}.
+     *
+     * @param external If {@code true} then this job instance resides in external process.
+     * @throws IgniteCheckedException If failed.
+     */
+    abstract public void dispose(boolean external) throws IgniteCheckedException;
+
+    /**
+     * Prepare local environment for the task.
+     *
+     * @param info Task info.
+     * @throws IgniteCheckedException If failed.
+     */
+    abstract public void prepareTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException;
+
+    /**
+     * Cleans up local environment of the task.
+     *
+     * @param info Task info.
+     * @throws IgniteCheckedException If failed.
+     */
+    abstract public void cleanupTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException;
+
+    /**
+     * Cleans up the job staging directory.
+     */
+    abstract public void cleanupStagingDirectory();
+
+    /**
+     * @return Ignite work directory.
+     */
+    abstract public String igniteWorkDirectory();
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String property(String name) {
+        return info().property(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasCombiner() {
+        return info().hasCombiner();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasReducer() {
+        return info().hasReducer();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int reducers() {
+        return info().reducers();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String jobName() {
+        return info().jobName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return info().user();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
index 853c63d..4cc8f80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
@@ -17,29 +17,29 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
-import java.io.Serializable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.jetbrains.annotations.Nullable;
 
 /**
- * Compact job description.
+ * Extended job description.
  */
-public interface HadoopJobInfo extends Serializable {
+public interface HadoopJobInfo {
     /**
      * Gets optional configuration property for the job.
      *
      * @param name Property name.
      * @return Value or {@code null} if none.
      */
-    @Nullable public String property(String name);
+    @Nullable String property(String name);
 
     /**
      * Checks whether job has combiner.
      *
      * @return {@code true} If job has combiner.
      */
-    public boolean hasCombiner();
+    boolean hasCombiner();
 
     /**
      * Checks whether job has reducer.
@@ -47,42 +47,42 @@ public interface HadoopJobInfo extends Serializable {
      *
      * @return Number of reducer.
      */
-    public boolean hasReducer();
-
-    /**
-     * Creates new job instance for the given ID.
-     * {@link HadoopJobInfo} is reusable for multiple jobs while {@link HadoopJob} is for one job execution.
-     * This method will be called once for the same ID on one node, though it can be called on the same host
-     * multiple times from different processes (in case of multiple nodes on the same host or external execution).
-     *
-     * @param jobCls The job class.
-     * @param jobId Job ID.
-     * @param log Logger.
-     * @param libNames Optional additional native library names.
-     * @param helper HadoopHelper.
-     * @return Job.
-     * @throws IgniteCheckedException If failed.
-     */
-    public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
-        HadoopJobId jobId, IgniteLogger log, @Nullable String[] libNames, HadoopHelper helper)
-            throws IgniteCheckedException;
+    boolean hasReducer();
 
     /**
      * @return Number of reducers configured for job.
      */
-    public int reducers();
+    int reducers();
 
     /**
      * Gets job name.
      *
      * @return Job name.
      */
-    public String jobName();
+    String jobName();
 
     /**
      * Gets user name.
      *
      * @return User name.
      */
-    public String user();
+    String user();
+
+    /**
+     * Creates new job instance for the given ID.
+     * {@link HadoopJobInfo} is reusable for multiple jobs while {@link HadoopJobEx} is for one job execution.
+     * This method will be called once for the same ID on one node, though it can be called on the same host
+     * multiple times from different processes (in case of multiple nodes on the same host or external execution).
+     *
+     * @param jobCls The job class.
+     * @param jobId Job ID.
+     * @param log Logger.
+     * @param libNames Optional additional native library names.
+     * @param helper HadoopHelper.
+     * @return Job.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopJobEx createJob(Class<? extends HadoopJobEx> jobCls,
+        HadoopJobId jobId, IgniteLogger log, @Nullable String[] libNames, HadoopHelper helper)
+            throws IgniteCheckedException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlan.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlan.java
deleted file mode 100644
index aadc2bf..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlan.java
+++ /dev/null
@@ -1,80 +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.hadoop;
-
-import java.io.Serializable;
-import java.util.Collection;
-import java.util.UUID;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Map-reduce job execution plan.
- */
-public interface HadoopMapReducePlan extends Serializable {
-    /**
-     * Gets collection of file blocks for which mappers should be executed.
-     *
-     * @param nodeId Node ID to check.
-     * @return Collection of file blocks or {@code null} if no mappers should be executed on given node.
-     */
-    @Nullable public Collection<HadoopInputSplit> mappers(UUID nodeId);
-
-    /**
-     * Gets reducer IDs that should be started on given node.
-     *
-     * @param nodeId Node ID to check.
-     * @return Array of reducer IDs.
-     */
-    @Nullable public int[] reducers(UUID nodeId);
-
-    /**
-     * Gets collection of all node IDs involved in map part of job execution.
-     *
-     * @return Collection of node IDs.
-     */
-    public Collection<UUID> mapperNodeIds();
-
-    /**
-     * Gets collection of all node IDs involved in reduce part of job execution.
-     *
-     * @return Collection of node IDs.
-     */
-    public Collection<UUID> reducerNodeIds();
-
-    /**
-     * Gets overall number of mappers for the job.
-     *
-     * @return Number of mappers.
-     */
-    public int mappers();
-
-    /**
-     * Gets overall number of reducers for the job.
-     *
-     * @return Number of reducers.
-     */
-    public int reducers();
-
-    /**
-     * Gets node ID for reducer.
-     *
-     * @param reducer Reducer.
-     * @return Node ID.
-     */
-    public UUID nodeForReducer(int reducer);
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.java
deleted file mode 100644
index 0009c4a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReducePlanner.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.internal.processors.hadoop;
-
-import java.util.Collection;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Map-reduce execution planner.
- */
-public interface HadoopMapReducePlanner {
-    /**
-     * Prepares map-reduce execution plan for the given job and topology.
-     *
-     * @param job Job.
-     * @param top Topology.
-     * @param oldPlan Old plan in case of partial failure.
-     * @return Map reduce plan.
-     * @throws IgniteCheckedException If an error occurs.
-     */
-    public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
-        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException;
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
index dddd017..194c1dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
@@ -29,7 +29,7 @@ import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawCompar
  */
 public abstract class HadoopTaskContext {
     /** */
-    protected final HadoopJob job;
+    protected final HadoopJobEx job;
 
     /** */
     private HadoopTaskInput input;
@@ -44,7 +44,7 @@ public abstract class HadoopTaskContext {
      * @param taskInfo Task info.
      * @param job Job.
      */
-    protected HadoopTaskContext(HadoopTaskInfo taskInfo, HadoopJob job) {
+    protected HadoopTaskContext(HadoopTaskInfo taskInfo, HadoopJobEx job) {
         this.taskInfo = taskInfo;
         this.job = job;
     }
@@ -88,7 +88,7 @@ public abstract class HadoopTaskContext {
     /**
      * @return Job.
      */
-    public HadoopJob job() {
+    public HadoopJobEx job() {
         return job;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
index 3509367..eb3113c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
@@ -21,6 +21,7 @@ import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
index 6c033b2..93a69db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.counter;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 
 /**
  * The object that writes some system counters to some storage for each running job. This operation is a part of
@@ -32,5 +32,5 @@ public interface HadoopCounterWriter {
      * @param cntrs Counters.
      * @throws IgniteCheckedException If failed.
      */
-    public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException;
+    public void write(HadoopJobEx job, HadoopCounters cntrs) throws IgniteCheckedException;
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 212e94a..02bad40 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1110,13 +1110,13 @@ org.apache.ignite.internal.processors.dr.GridDrType
 org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater
 org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo
 org.apache.ignite.internal.processors.hadoop.HadoopFileBlock
-org.apache.ignite.internal.processors.hadoop.HadoopInputSplit
+org.apache.ignite.hadoop.HadoopInputSplit
 org.apache.ignite.internal.processors.hadoop.HadoopJobId
 org.apache.ignite.internal.processors.hadoop.HadoopJobInfo
 org.apache.ignite.internal.processors.hadoop.HadoopJobPhase
 org.apache.ignite.internal.processors.hadoop.HadoopJobProperty
 org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
-org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan
+org.apache.ignite.hadoop.HadoopMapReducePlan
 org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo
 org.apache.ignite.internal.processors.hadoop.HadoopTaskType
 org.apache.ignite.internal.processors.hadoop.message.HadoopMessage

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
index f1c1b16..1128fa4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.hadoop.fs;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterWriter;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.delegate.HadoopDelegateUtils;
@@ -41,7 +41,7 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     private volatile HadoopFileSystemCounterWriterDelegate delegate;
 
     /** {@inheritDoc} */
-    @Override public void write(HadoopJob job, HadoopCounters cntrs)
+    @Override public void write(HadoopJobEx job, HadoopCounters cntrs)
         throws IgniteCheckedException {
         delegate(job).write(job, cntrs);
     }
@@ -52,7 +52,7 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
      * @param job Job.
      * @return Delegate.
      */
-    private HadoopFileSystemCounterWriterDelegate delegate(HadoopJob job) {
+    private HadoopFileSystemCounterWriterDelegate delegate(HadoopJobEx job) {
         HadoopFileSystemCounterWriterDelegate delegate0 = delegate;
 
         if (delegate0 == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java
index 178cdb5..bb0b47f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopWeightedMapReducePlanner.java
@@ -21,16 +21,16 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.hadoop.HadoopJob;
 import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsEndpoint;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner;
+import org.apache.ignite.hadoop.planner.HadoopAbstractMapReducePlanner;
 import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.planner.HadoopMapReducePlanGroup;
 import org.apache.ignite.internal.processors.hadoop.planner.HadoopMapReducePlanTopology;
@@ -117,7 +117,7 @@ public class IgniteHadoopWeightedMapReducePlanner extends HadoopAbstractMapReduc
     @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> nodes,
         @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException {
         List<HadoopInputSplit> splits = HadoopCommonUtils.sortInputSplits(job.input());
-        int reducerCnt = job.info().reducers();
+        int reducerCnt = job.reducers();
 
         if (reducerCnt < 0)
             throw new IgniteCheckedException("Number of reducers must be non-negative, actual: " + reducerCnt);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopAbstractMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopAbstractMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopAbstractMapReducePlanner.java
new file mode 100644
index 0000000..dd01f11
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopAbstractMapReducePlanner.java
@@ -0,0 +1,118 @@
+/*
+ * 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.hadoop.planner;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.hadoop.HadoopMapReducePlanner;
+import org.apache.ignite.internal.processors.hadoop.planner.HadoopMapReducePlanGroup;
+import org.apache.ignite.internal.processors.hadoop.planner.HadoopMapReducePlanTopology;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.resources.LoggerResource;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.UUID;
+
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
+
+/**
+ * Base class for map-reduce planners.
+ */
+public abstract class HadoopAbstractMapReducePlanner implements HadoopMapReducePlanner {
+    /** Injected grid. */
+    @IgniteInstanceResource
+    protected Ignite ignite;
+
+    /** Logger. */
+    @SuppressWarnings("UnusedDeclaration")
+    @LoggerResource
+    protected IgniteLogger log;
+
+    /**
+     * Create plan topology.
+     *
+     * @param nodes Topology nodes.
+     * @return Plan topology.
+     */
+    protected static HadoopMapReducePlanTopology topology(Collection<ClusterNode> nodes) {
+        Map<String, HadoopMapReducePlanGroup> macsMap = new HashMap<>(nodes.size());
+
+        Map<UUID, HadoopMapReducePlanGroup> idToGrp = new HashMap<>(nodes.size());
+        Map<String, HadoopMapReducePlanGroup> hostToGrp = new HashMap<>(nodes.size());
+
+        for (ClusterNode node : nodes) {
+            String macs = node.attribute(ATTR_MACS);
+
+            HadoopMapReducePlanGroup grp = macsMap.get(macs);
+
+            if (grp == null) {
+                grp = new HadoopMapReducePlanGroup(node, macs);
+
+                macsMap.put(macs, grp);
+            }
+            else
+                grp.add(node);
+
+            idToGrp.put(node.id(), grp);
+
+            for (String host : node.addresses()) {
+                HadoopMapReducePlanGroup hostGrp = hostToGrp.get(host);
+
+                if (hostGrp == null)
+                    hostToGrp.put(host, grp);
+                else
+                    assert hostGrp == grp;
+            }
+        }
+
+        return new HadoopMapReducePlanTopology(new ArrayList<>(macsMap.values()), idToGrp, hostToGrp);
+    }
+
+
+    /**
+     * Groups nodes by host names.
+     *
+     * @param top Topology to group.
+     * @return Map.
+     */
+    protected static Map<String, Collection<UUID>> groupByHost(Collection<ClusterNode> top) {
+        Map<String, Collection<UUID>> grouped = U.newHashMap(top.size());
+
+        for (ClusterNode node : top) {
+            for (String host : node.hostNames()) {
+                Collection<UUID> nodeIds = grouped.get(host);
+
+                if (nodeIds == null) {
+                    // Expecting 1-2 nodes per host.
+                    nodeIds = new ArrayList<>(2);
+
+                    grouped.put(host, nodeIds);
+                }
+
+                nodeIds.add(node.id());
+            }
+        }
+
+        return grouped;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopTestRoundRobinMrPlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopTestRoundRobinMrPlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopTestRoundRobinMrPlanner.java
new file mode 100644
index 0000000..d9de0c1
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/planner/HadoopTestRoundRobinMrPlanner.java
@@ -0,0 +1,75 @@
+/*
+ * 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.hadoop.planner;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlanner;
+import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Round-robin mr planner.
+ */
+public class HadoopTestRoundRobinMrPlanner implements HadoopMapReducePlanner {
+    /** {@inheritDoc} */
+    @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
+        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException {
+        if (top.isEmpty())
+            throw new IllegalArgumentException("Topology is empty");
+
+        // Has at least one element.
+        Iterator<ClusterNode> it = top.iterator();
+
+        Map<UUID, Collection<HadoopInputSplit>> mappers = new HashMap<>();
+
+        for (HadoopInputSplit block : job.input()) {
+            ClusterNode node = it.next();
+
+            Collection<HadoopInputSplit> nodeBlocks = mappers.get(node.id());
+
+            if (nodeBlocks == null) {
+                nodeBlocks = new ArrayList<>();
+
+                mappers.put(node.id(), nodeBlocks);
+            }
+
+            nodeBlocks.add(block);
+
+            if (!it.hasNext())
+                it = top.iterator();
+        }
+
+        int[] rdc = new int[job.reducers()];
+
+        for (int i = 0; i < rdc.length; i++)
+            rdc[i] = i;
+
+        return new HadoopDefaultMapReducePlan(mappers, Collections.singletonMap(it.next().id(), rdc));
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java
index 37af147..7e74d82 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopCommonUtils.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.ArrayList;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
index 4326ad2..f125485 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopContext.java
@@ -23,6 +23,8 @@ import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.HadoopConfiguration;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlanner;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java
index a9b4532..7db535a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopExternalSplit.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop;
 
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
 import java.io.Externalizable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java
index fb6d0f3..fe5d434 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopSplitWrapper.java
@@ -22,6 +22,7 @@ import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.util.Arrays;
 
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java
index 541cf80..0d82b5b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/delegate/HadoopFileSystemCounterWriterDelegate.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.delegate;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 
 /**
@@ -32,5 +32,5 @@ public interface HadoopFileSystemCounterWriterDelegate {
      * @param cntrs Counters.
      * @throws IgniteCheckedException If failed.
      */
-    public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException;
+    public void write(HadoopJobEx job, HadoopCounters cntrs) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java
index d4c10da..6b36d26 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/delegate/HadoopFileSystemCounterWriterDelegateImpl.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.hadoop.fs.IgniteHadoopFileSystemCounterWriter;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
@@ -60,7 +60,7 @@ public class HadoopFileSystemCounterWriterDelegateImpl implements HadoopFileSyst
     }
 
     /** {@inheritDoc} */
-    public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException {
+    public void write(HadoopJobEx job, HadoopCounters cntrs) throws IgniteCheckedException {
         Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration();
 
         final HadoopJobInfo jobInfo = job.info();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java
index 65ff280..cde6da6 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1MapTask.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
@@ -54,7 +54,7 @@ public class HadoopV1MapTask extends HadoopV1Task {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void run(HadoopTaskContext taskCtx) throws IgniteCheckedException {
-        HadoopJob job = taskCtx.job();
+        HadoopJobEx job = taskCtx.job();
 
         HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java
index 92c024e..6b90653 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1ReduceTask.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.mapred.Reducer;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
@@ -51,7 +51,7 @@ public class HadoopV1ReduceTask extends HadoopV1Task {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void run(HadoopTaskContext taskCtx) throws IgniteCheckedException {
-        HadoopJob job = taskCtx.job();
+        HadoopJobEx job = taskCtx.job();
 
         HadoopV2TaskContext ctx = (HadoopV2TaskContext)taskCtx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java
index 11a3598..26325b9 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v1/HadoopV1Splitter.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.mapred.InputSplit;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.jetbrains.annotations.Nullable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
index eec0636..11f2ecc 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
index a24e581..5d3f22d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.split.JobSplit;
 import org.apache.hadoop.mapreduce.split.SplitMetaInfoReader;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
@@ -37,8 +38,8 @@ import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
@@ -85,7 +86,7 @@ import static org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSys
 /**
  * Hadoop job implementation for v2 API.
  */
-public class HadoopV2Job implements HadoopJob {
+public class HadoopV2Job extends HadoopJobEx {
     /** */
     private final JobConf jobConf;
 
@@ -139,6 +140,7 @@ public class HadoopV2Job implements HadoopJob {
      * @param jobInfo Job info.
      * @param log Logger.
      * @param libNames Optional additional native library names.
+     * @param helper Hadoop helper.
      */
     public HadoopV2Job(HadoopJobId jobId, final HadoopDefaultJobInfo jobInfo, IgniteLogger log,
         @Nullable String[] libNames, HadoopHelper helper) {
@@ -182,7 +184,7 @@ public class HadoopV2Job implements HadoopJob {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<HadoopInputSplit> input() throws IgniteCheckedException {
+    @Override public Collection<HadoopInputSplit> input() {
         ClassLoader oldLdr = HadoopCommonUtils.setContextClassLoader(jobConf.getClassLoader());
 
         try {
@@ -239,6 +241,9 @@ public class HadoopV2Job implements HadoopJob {
                     throw transformException(e);
             }
         }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
         finally {
             HadoopCommonUtils.restoreContextClassLoader(oldLdr);
         }
@@ -274,7 +279,7 @@ public class HadoopV2Job implements HadoopJob {
                 fullCtxClsQueue.add(cls);
             }
 
-            Constructor<?> ctr = cls.getConstructor(HadoopTaskInfo.class, HadoopJob.class,
+            Constructor<?> ctr = cls.getConstructor(HadoopTaskInfo.class, HadoopJobEx.class,
                 HadoopJobId.class, UUID.class, DataInput.class);
 
             if (jobConfData == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java
index 667ef1e..c878515 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Splitter.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
 import org.jetbrains.annotations.Nullable;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index d328550..8b8a728 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -45,8 +45,8 @@ import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
 import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
@@ -165,7 +165,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
      * @param locNodeId Local node ID.
      * @param jobConfDataInput DataInput for read JobConf.
      */
-    public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJob job, HadoopJobId jobId,
+    public HadoopV2TaskContext(HadoopTaskInfo taskInfo, HadoopJobEx job, HadoopJobId jobId,
         @Nullable UUID locNodeId, DataInput jobConfDataInput) throws IgniteCheckedException {
         super(taskInfo, job);
         this.locNodeId = locNodeId;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
index 090b336..1035701 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobMetadata.java
@@ -24,11 +24,11 @@ import java.io.ObjectOutput;
 import java.util.Collection;
 import java.util.Map;
 import java.util.UUID;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 9542372..9284c02 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -49,14 +49,14 @@ import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopComponent;
 import org.apache.ignite.internal.processors.hadoop.HadoopContext;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobStatus;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlanner;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterWriter;
@@ -114,7 +114,7 @@ public class HadoopJobTracker extends HadoopComponent {
     private HadoopMapReducePlanner mrPlanner;
 
     /** All the known jobs. */
-    private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJob>> jobs = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<HadoopJobId, GridFutureAdapter<HadoopJobEx>> jobs = new ConcurrentHashMap8<>();
 
     /** Locally active jobs. */
     private final ConcurrentMap<HadoopJobId, JobLocalState> activeJobs = new ConcurrentHashMap8<>();
@@ -129,8 +129,8 @@ public class HadoopJobTracker extends HadoopComponent {
     /** Component busy lock. */
     private GridSpinReadWriteLock busyLock;
 
-    /** Class to create HadoopJob instances from. */
-    private Class<? extends HadoopJob> jobCls;
+    /** Class to create HadoopJobEx instances from. */
+    private Class<? extends HadoopJobEx> jobCls;
 
     /** Closure to check result of async transform of system cache. */
     private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
@@ -158,7 +158,7 @@ public class HadoopJobTracker extends HadoopComponent {
         HadoopClassLoader ldr = ctx.kernalContext().hadoopHelper().commonClassLoader();
 
         try {
-            jobCls = (Class<HadoopJob>)ldr.loadClass(HadoopCommonUtils.JOB_CLS_NAME);
+            jobCls = (Class<HadoopJobEx>)ldr.loadClass(HadoopCommonUtils.JOB_CLS_NAME);
         }
         catch (Exception ioe) {
             throw new IgniteCheckedException("Failed to load job class [class=" +
@@ -310,7 +310,7 @@ public class HadoopJobTracker extends HadoopComponent {
             if (jobs.containsKey(jobId) || jobMetaCache().containsKey(jobId))
                 throw new IgniteCheckedException("Failed to submit job. Job with the same ID already exists: " + jobId);
 
-            HadoopJob job = job(jobId, info);
+            HadoopJobEx job = job(jobId, info);
 
             HadoopMapReducePlan mrPlan = mrPlanner.preparePlan(job, ctx.nodes(), null);
 
@@ -692,7 +692,7 @@ public class HadoopJobTracker extends HadoopComponent {
                 try {
                     if (checkSetup && phase == PHASE_SETUP && !activeJobs.containsKey(jobId)) {
                         // Failover setup task.
-                        HadoopJob job = job(jobId, meta.jobInfo());
+                        HadoopJobEx job = job(jobId, meta.jobInfo());
 
                         Collection<HadoopTaskInfo> setupTask = setupTask(jobId);
 
@@ -818,7 +818,7 @@ public class HadoopJobTracker extends HadoopComponent {
         throws IgniteCheckedException {
         JobLocalState state = activeJobs.get(jobId);
 
-        HadoopJob job = job(jobId, meta.jobInfo());
+        HadoopJobEx job = job(jobId, meta.jobInfo());
 
         HadoopMapReducePlan plan = meta.mapReducePlan();
 
@@ -1048,7 +1048,7 @@ public class HadoopJobTracker extends HadoopComponent {
      * @param job Job instance.
      * @return Collection of task infos.
      */
-    private Collection<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJob job) {
+    private Collection<HadoopTaskInfo> reducerTasks(int[] reducers, HadoopJobEx job) {
         UUID locNodeId = ctx.localNodeId();
         HadoopJobId jobId = job.id();
 
@@ -1097,15 +1097,15 @@ public class HadoopJobTracker extends HadoopComponent {
      * @return Job.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public HadoopJob job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException {
-        GridFutureAdapter<HadoopJob> fut = jobs.get(jobId);
+    @Nullable public HadoopJobEx job(HadoopJobId jobId, @Nullable HadoopJobInfo jobInfo) throws IgniteCheckedException {
+        GridFutureAdapter<HadoopJobEx> fut = jobs.get(jobId);
 
-        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapter<HadoopJob>())) != null)
+        if (fut != null || (fut = jobs.putIfAbsent(jobId, new GridFutureAdapter<HadoopJobEx>())) != null)
             return fut.get();
 
         fut = jobs.get(jobId);
 
-        HadoopJob job = null;
+        HadoopJobEx job = null;
 
         try {
             if (jobInfo == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopAbstractMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopAbstractMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopAbstractMapReducePlanner.java
deleted file mode 100644
index f01f72b..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopAbstractMapReducePlanner.java
+++ /dev/null
@@ -1,116 +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.hadoop.planner;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.resources.IgniteInstanceResource;
-import org.apache.ignite.resources.LoggerResource;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.UUID;
-
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
-
-/**
- * Base class for map-reduce planners.
- */
-public abstract class HadoopAbstractMapReducePlanner implements HadoopMapReducePlanner {
-    /** Injected grid. */
-    @IgniteInstanceResource
-    protected Ignite ignite;
-
-    /** Logger. */
-    @SuppressWarnings("UnusedDeclaration")
-    @LoggerResource
-    protected IgniteLogger log;
-
-    /**
-     * Create plan topology.
-     *
-     * @param nodes Topology nodes.
-     * @return Plan topology.
-     */
-    protected static HadoopMapReducePlanTopology topology(Collection<ClusterNode> nodes) {
-        Map<String, HadoopMapReducePlanGroup> macsMap = new HashMap<>(nodes.size());
-
-        Map<UUID, HadoopMapReducePlanGroup> idToGrp = new HashMap<>(nodes.size());
-        Map<String, HadoopMapReducePlanGroup> hostToGrp = new HashMap<>(nodes.size());
-
-        for (ClusterNode node : nodes) {
-            String macs = node.attribute(ATTR_MACS);
-
-            HadoopMapReducePlanGroup grp = macsMap.get(macs);
-
-            if (grp == null) {
-                grp = new HadoopMapReducePlanGroup(node, macs);
-
-                macsMap.put(macs, grp);
-            }
-            else
-                grp.add(node);
-
-            idToGrp.put(node.id(), grp);
-
-            for (String host : node.addresses()) {
-                HadoopMapReducePlanGroup hostGrp = hostToGrp.get(host);
-
-                if (hostGrp == null)
-                    hostToGrp.put(host, grp);
-                else
-                    assert hostGrp == grp;
-            }
-        }
-
-        return new HadoopMapReducePlanTopology(new ArrayList<>(macsMap.values()), idToGrp, hostToGrp);
-    }
-
-
-    /**
-     * Groups nodes by host names.
-     *
-     * @param top Topology to group.
-     * @return Map.
-     */
-    protected static Map<String, Collection<UUID>> groupByHost(Collection<ClusterNode> top) {
-        Map<String, Collection<UUID>> grouped = U.newHashMap(top.size());
-
-        for (ClusterNode node : top) {
-            for (String host : node.hostNames()) {
-                Collection<UUID> nodeIds = grouped.get(host);
-
-                if (nodeIds == null) {
-                    // Expecting 1-2 nodes per host.
-                    nodeIds = new ArrayList<>(2);
-
-                    grouped.put(host, nodeIds);
-                }
-
-                nodeIds.add(node.id());
-            }
-        }
-
-        return grouped;
-    }
-}


[31/50] [abbrv] ignite git commit: IGNITE-4428: Hadoop: moved HadoopMapReducePlanner and dependent classes to public space. This closes #1389. This closes #1394.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
index 7aaf3fa..1004e3c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/HadoopDefaultMapReducePlan.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.hadoop.planner;
 
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 8ffea8c..6f9b7a4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -25,9 +25,9 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.processors.hadoop.HadoopComponent;
 import org.apache.ignite.internal.processors.hadoop.HadoopContext;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 318ead3..25925fc 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -21,7 +21,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapperUtils;
@@ -86,7 +86,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     private static final boolean DFLT_SHUFFLE_MSG_GZIP = false;
 
     /** */
-    private final HadoopJob job;
+    private final HadoopJobEx job;
 
     /** */
     private final GridUnsafeMemory mem;
@@ -169,7 +169,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
      * @param embedded Whether shuffle is running in embedded mode.
      * @throws IgniteCheckedException If error.
      */
-    public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, HadoopJob job, GridUnsafeMemory mem,
+    public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, HadoopJobEx job, GridUnsafeMemory mem,
         int totalReducerCnt, int[] locReducers, int locMappersCnt, boolean embedded) throws IgniteCheckedException {
         this.locReduceAddr = locReduceAddr;
         this.totalReducerCnt = totalReducerCnt;
@@ -412,6 +412,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
      *
      * @param msg Message.
      * @return Buffer.
+     * @throws IgniteCheckedException On error.
      */
     private byte[] extractBuffer(HadoopDirectShuffleMessage msg) throws IgniteCheckedException {
         if (msgGzip) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
index 5ede18e..96893b1 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopEmbeddedTaskExecutor.java
@@ -21,7 +21,7 @@ import java.util.Collection;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
@@ -76,7 +76,7 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void run(final HadoopJob job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException {
+    @Override public void run(final HadoopJobEx job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Submitting tasks for local execution [locNodeId=" + ctx.localNodeId() +
                 ", tasksCnt=" + tasks.size() + ']');
@@ -128,8 +128,8 @@ public class HadoopEmbeddedTaskExecutor extends HadoopTaskExecutorAdapter {
      * for this job ID.
      * <p>
      * It is guaranteed that this method will not be called concurrently with
-     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via
-     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called.
+     * {@link #run(HadoopJobEx, Collection)} method. No more job submissions will be performed via
+     * {@link #run(HadoopJobEx, Collection)} method for given job ID after this method is called.
      *
      * @param jobId Job ID to cancel.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
index a57efe6..afa01a4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
@@ -21,7 +21,7 @@ import java.util.UUID;
 import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
@@ -51,7 +51,7 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
     private final IgniteLogger log;
 
     /** */
-    private final HadoopJob job;
+    private final HadoopJobEx job;
 
     /** Task to run. */
     private final HadoopTaskInfo info;
@@ -84,7 +84,7 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
      * @param info Task info.
      * @param nodeId Node id.
      */
-    protected HadoopRunnableTask(IgniteLogger log, HadoopJob job, GridUnsafeMemory mem, HadoopTaskInfo info,
+    protected HadoopRunnableTask(IgniteLogger log, HadoopJobEx job, GridUnsafeMemory mem, HadoopTaskInfo info,
         UUID nodeId) {
         this.nodeId = nodeId;
         this.log = log.getLogger(HadoopRunnableTask.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
index f13c76a..b7d0a34 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopTaskExecutorAdapter.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.hadoop.taskexecutor;
 import java.util.Collection;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopComponent;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata;
@@ -36,15 +36,15 @@ public abstract class HadoopTaskExecutorAdapter extends HadoopComponent {
      * @param tasks Tasks.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void run(final HadoopJob job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException;
+    public abstract void run(final HadoopJobEx job, Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException;
 
     /**
      * Cancels all currently running tasks for given job ID and cancels scheduled execution of tasks
      * for this job ID.
      * <p>
      * It is guaranteed that this method will not be called concurrently with
-     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method. No more job submissions will be performed via
-     * {@link #run(org.apache.ignite.internal.processors.hadoop.HadoopJob, Collection)} method for given job ID after this method is called.
+     * {@link #run(HadoopJobEx, Collection)} method. No more job submissions will be performed via
+     * {@link #run(HadoopJobEx, Collection)} method for given job ID after this method is called.
      *
      * @param jobId Job ID to cancel.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
index 2c560bc..afa3e25 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/HadoopExternalTaskExecutor.java
@@ -33,10 +33,10 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.HadoopContext;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobPhase;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskType;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata;
@@ -198,7 +198,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
             }
         }
         else if (ctx.isParticipating(meta)) {
-            HadoopJob job;
+            HadoopJobEx job;
 
             try {
                 job = jobTracker.job(meta.jobId(), meta.jobInfo());
@@ -215,7 +215,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("ConstantConditions")
-    @Override public void run(final HadoopJob job, final Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException {
+    @Override public void run(final HadoopJobEx job, final Collection<HadoopTaskInfo> tasks) throws IgniteCheckedException {
         if (!busyLock.tryReadLock()) {
             if (log.isDebugEnabled())
                 log.debug("Failed to start hadoop tasks (grid is stopping, will ignore).");
@@ -293,7 +293,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param job Job instance.
      * @param tasks Collection of tasks to execute in started process.
      */
-    private void sendExecutionRequest(HadoopProcess proc, HadoopJob job, Collection<HadoopTaskInfo> tasks)
+    private void sendExecutionRequest(HadoopProcess proc, HadoopJobEx job, Collection<HadoopTaskInfo> tasks)
         throws IgniteCheckedException {
         // Must synchronize since concurrent process crash may happen and will receive onConnectionLost().
         proc.lock();
@@ -349,7 +349,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param job Job instance.
      * @param plan Map reduce plan.
      */
-    private HadoopProcess startProcess(final HadoopJob job, final HadoopMapReducePlan plan) {
+    private HadoopProcess startProcess(final HadoopJobEx job, final HadoopMapReducePlan plan) {
         final UUID childProcId = UUID.randomUUID();
 
         HadoopJobId jobId = job.id();
@@ -523,7 +523,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @return Started process.
      */
     private Process startJavaProcess(UUID childProcId, HadoopExternalTaskMetadata startMeta,
-        HadoopJob job, String igniteWorkDir) throws Exception {
+        HadoopJobEx job, String igniteWorkDir) throws Exception {
         String outFldr = jobWorkFolder(job.id()) + File.separator + childProcId;
 
         if (log.isDebugEnabled())
@@ -633,7 +633,7 @@ public class HadoopExternalTaskExecutor extends HadoopTaskExecutorAdapter {
      * @param job Job.
      * @param plan Map reduce plan.
      */
-    private void prepareForJob(HadoopProcess proc, HadoopJob job, HadoopMapReducePlan plan) {
+    private void prepareForJob(HadoopProcess proc, HadoopJobEx job, HadoopMapReducePlan plan) {
         try {
             comm.sendMessage(proc.descriptor(), new HadoopPrepareForJobRequest(job.id(), job.info(),
                 plan.reducers(), plan.reducers(ctx.localNodeId())));

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index 3336120..8d23ffe 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -26,7 +26,7 @@ import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
@@ -90,7 +90,7 @@ public class HadoopChildProcessRunner {
     private final GridFutureAdapter<?> initFut = new GridFutureAdapter<>();
 
     /** Job instance. */
-    private HadoopJob job;
+    private HadoopJobEx job;
 
     /** Number of uncompleted tasks. */
     private final AtomicInteger pendingTasks = new AtomicInteger();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/resources/META-INF/classnames.properties b/modules/hadoop/src/main/resources/META-INF/classnames.properties
index 0ac17cf..f528107 100644
--- a/modules/hadoop/src/main/resources/META-INF/classnames.properties
+++ b/modules/hadoop/src/main/resources/META-INF/classnames.properties
@@ -29,13 +29,13 @@ org.apache.ignite.internal.processors.hadoop.HadoopAttributes
 org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo
 org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit
 org.apache.ignite.internal.processors.hadoop.HadoopFileBlock
-org.apache.ignite.internal.processors.hadoop.HadoopInputSplit
+org.apache.ignite.hadoop.HadoopInputSplit
 org.apache.ignite.internal.processors.hadoop.HadoopJobId
 org.apache.ignite.internal.processors.hadoop.HadoopJobInfo
 org.apache.ignite.internal.processors.hadoop.HadoopJobPhase
 org.apache.ignite.internal.processors.hadoop.HadoopJobProperty
 org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
-org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan
+org.apache.ignite.hadoop.HadoopMapReducePlan
 org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper
 org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException
 org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java
index 0be8bf9..666acdf 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopCommandLineTest.java
@@ -38,7 +38,7 @@ import org.apache.ignite.igfs.IgfsInputStream;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgnitionEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
@@ -245,7 +245,7 @@ public class HadoopCommandLineTest extends GridCommonAbstractTest {
     private ProcessBuilder createProcessBuilder() {
         String sep = ":";
 
-        String ggClsPath = HadoopJob.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
+        String ggClsPath = HadoopJobEx.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
             HadoopJobTracker.class.getProtectionDomain().getCodeSource().getLocation().getPath() + sep +
             ConcurrentHashMap8.class.getProtectionDomain().getCodeSource().getLocation().getPath();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
index 91ad5ec..692c2a3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
@@ -36,6 +36,7 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.configuration.HadoopConfiguration;
+import org.apache.ignite.hadoop.planner.HadoopTestRoundRobinMrPlanner;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java
index 7e6fa9c..28c8264 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopPlannerMockJob.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.hadoop.impl;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
@@ -34,7 +34,7 @@ import java.util.UUID;
 /**
  * Mock job for planner tests.
  */
-public class HadoopPlannerMockJob implements HadoopJob {
+public class HadoopPlannerMockJob extends HadoopJobEx {
     /** Input splits. */
     private final Collection<HadoopInputSplit> splits;
 
@@ -53,7 +53,7 @@ public class HadoopPlannerMockJob implements HadoopJob {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<HadoopInputSplit> input() throws IgniteCheckedException {
+    @Override public Collection<HadoopInputSplit> input() {
         return splits;
     }
 
@@ -158,7 +158,7 @@ public class HadoopPlannerMockJob implements HadoopJob {
         }
 
         /** {@inheritDoc} */
-        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls, HadoopJobId jobId, IgniteLogger log,
+        @Override public HadoopJobEx createJob(Class<? extends HadoopJobEx> jobCls, HadoopJobId jobId, IgniteLogger log,
             @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException {
             throwUnsupported();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java
index 8b1b693..5d34989 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksAllVersionsTest.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskType;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2;
@@ -48,7 +48,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    public abstract HadoopJob getHadoopJob(String inFile, String outFile) throws Exception;
+    public abstract HadoopJobEx getHadoopJob(String inFile, String outFile) throws Exception;
 
     /**
      * @return prefix of reducer output file name. It's "part-" for v1 and "part-r-" for v2 API
@@ -84,7 +84,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, fileBlock1.length(),
                 igfs.info(inFile).length() - fileBlock1.length());
 
-        HadoopJob gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJobEx gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock1);
 
@@ -115,7 +115,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runTaskWithInput(HadoopJob gridJob, HadoopTaskType taskType,
+    private HadoopTestTaskContext runTaskWithInput(HadoopJobEx gridJob, HadoopTaskType taskType,
         int taskNum, String... words) throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(taskType, gridJob.id(), taskNum, 0, null);
 
@@ -141,7 +141,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testReduceTask() throws Exception {
-        HadoopJob gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
+        HadoopJobEx gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
 
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 0, "word1", "5", "word2", "10");
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 1, "word3", "7", "word4", "15");
@@ -167,7 +167,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testCombinerTask() throws Exception {
-        HadoopJob gridJob = getHadoopJob("/", "/");
+        HadoopJobEx gridJob = getHadoopJob("/", "/");
 
         HadoopTestTaskContext ctx =
             runTaskWithInput(gridJob, HadoopTaskType.COMBINE, 0, "word1", "5", "word2", "10");
@@ -187,7 +187,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context of combine task with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopJob gridJob)
+    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopJobEx gridJob)
         throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock);
 
@@ -233,7 +233,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock1 = new HadoopFileBlock(HOSTS, inFileUri, 0, l);
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, l, fileLen - l);
 
-        HadoopJob gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJobEx gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java
index d7cd738..1d7f3e4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV1Test.java
@@ -21,7 +21,7 @@ import java.io.IOException;
 import java.util.UUID;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1;
@@ -41,7 +41,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJobEx getHadoopJob(String inFile, String outFile) throws Exception {
         JobConf jobConf = HadoopWordCount1.getJob(inFile, outFile);
 
         setupFileSystems(jobConf);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java
index c635c41..61e3e46 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTasksV2Test.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.Job;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelperImpl;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2;
@@ -46,7 +46,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws Exception if fails.
      */
-    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJobEx getHadoopJob(String inFile, String outFile) throws Exception {
         Job job = Job.getInstance();
 
         job.setOutputKeyClass(Text.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java
deleted file mode 100644
index 81f6f3c..0000000
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestRoundRobinMrPlanner.java
+++ /dev/null
@@ -1,75 +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.hadoop.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlanner;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Round-robin mr planner.
- */
-public class HadoopTestRoundRobinMrPlanner implements HadoopMapReducePlanner {
-    /** {@inheritDoc} */
-    @Override public HadoopMapReducePlan preparePlan(HadoopJob job, Collection<ClusterNode> top,
-        @Nullable HadoopMapReducePlan oldPlan) throws IgniteCheckedException {
-        if (top.isEmpty())
-            throw new IllegalArgumentException("Topology is empty");
-
-        // Has at least one element.
-        Iterator<ClusterNode> it = top.iterator();
-
-        Map<UUID, Collection<HadoopInputSplit>> mappers = new HashMap<>();
-
-        for (HadoopInputSplit block : job.input()) {
-            ClusterNode node = it.next();
-
-            Collection<HadoopInputSplit> nodeBlocks = mappers.get(node.id());
-
-            if (nodeBlocks == null) {
-                nodeBlocks = new ArrayList<>();
-
-                mappers.put(node.id(), nodeBlocks);
-            }
-
-            nodeBlocks.add(block);
-
-            if (!it.hasNext())
-                it = top.iterator();
-        }
-
-        int[] rdc = new int[job.info().reducers()];
-
-        for (int i = 0; i < rdc.length; i++)
-            rdc[i] = i;
-
-        return new HadoopDefaultMapReducePlan(mappers, Collections.singletonMap(it.next().id(), rdc));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java
index cfd41cf..fd213a1 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTestTaskContext.java
@@ -33,7 +33,7 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskOutput;
@@ -192,7 +192,7 @@ class HadoopTestTaskContext extends HadoopV2TaskContext {
      * @param taskInfo Task info.
      * @param gridJob Grid Hadoop job.
      */
-    public HadoopTestTaskContext(HadoopTaskInfo taskInfo, HadoopJob gridJob) throws IgniteCheckedException {
+    public HadoopTestTaskContext(HadoopTaskInfo taskInfo, HadoopJobEx gridJob) throws IgniteCheckedException {
         super(taskInfo, gridJob, gridJob.id(), null, jobConfDataInput(gridJob));
     }
 
@@ -203,7 +203,7 @@ class HadoopTestTaskContext extends HadoopV2TaskContext {
      * @return DataInput with JobConf.
      * @throws IgniteCheckedException If failed.
      */
-    private static DataInput jobConfDataInput(HadoopJob job) throws IgniteCheckedException {
+    private static DataInput jobConfDataInput(HadoopJobEx job) throws IgniteCheckedException {
         JobConf jobConf = new JobConf();
 
         for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)job.info()).properties().entrySet())

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java
index 540a7aa..2c2f049 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopV2JobSelfTest.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.io.serializer.WritableSerialization;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
@@ -69,7 +69,7 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
     }
 
     /**
-     * Tests that {@link HadoopJob} provides wrapped serializer if it's set in configuration.
+     * Tests that {@link HadoopJobEx} provides wrapped serializer if it's set in configuration.
      *
      * @throws IgniteCheckedException If fails.
      */
@@ -86,7 +86,7 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
 
         HadoopJobId id = new HadoopJobId(uuid, 1);
 
-        HadoopJob job = info.createJob(HadoopV2Job.class, id, log, null, new HadoopHelperImpl());
+        HadoopJobEx job = info.createJob(HadoopV2Job.class, id, log, null, new HadoopHelperImpl());
 
         HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
             null));

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java
index 430c675..6dcd998 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopWeightedMapReducePlannerTest.java
@@ -24,9 +24,9 @@ import org.apache.ignite.igfs.IgfsBlockLocation;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgniteNodeAttributes;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
-import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
-import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
-import org.apache.ignite.internal.processors.hadoop.planner.HadoopAbstractMapReducePlanner;
+import org.apache.ignite.hadoop.HadoopInputSplit;
+import org.apache.ignite.hadoop.HadoopMapReducePlan;
+import org.apache.ignite.hadoop.planner.HadoopAbstractMapReducePlanner;
 import org.apache.ignite.internal.processors.igfs.IgfsIgniteMock;
 import org.apache.ignite.internal.processors.igfs.IgfsMock;
 import org.apache.ignite.internal.util.typedef.F;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d14e0727/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
index 1f8978d..b9dcae1 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
@@ -24,7 +24,7 @@ import org.apache.hadoop.io.IntWritable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.processors.hadoop.HadoopHelper;
-import org.apache.ignite.internal.processors.hadoop.HadoopJob;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
@@ -150,7 +150,7 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls, HadoopJobId jobId, IgniteLogger log,
+        @Override public HadoopJobEx createJob(Class<? extends HadoopJobEx> jobCls, HadoopJobId jobId, IgniteLogger log,
             @Nullable String[] libNames, HadoopHelper helper) throws IgniteCheckedException {
             assert false;
 


[34/50] [abbrv] ignite git commit: IGNITE-4503: Hadoop: added boundary checks to HadoopDirectDataInput. This closes # 1416.

Posted by sb...@apache.org.
IGNITE-4503: Hadoop: added boundary checks to HadoopDirectDataInput. This closes # 1416.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: f1365421c299b754a10edf8b6f156aeeb5ff0ce1
Parents: d14e072
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Mon Jan 16 16:57:27 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jan 16 16:57:27 2017 +0300

----------------------------------------------------------------------
 .../shuffle/direct/HadoopDirectDataInput.java   | 41 +++++++++++++++++++-
 1 file changed, 39 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f1365421/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
index 6f0e2b0..8031c9f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
 
+import java.io.EOFException;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.jetbrains.annotations.NotNull;
@@ -59,6 +60,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public void readFully(@NotNull byte[] b, int off, int len) throws IOException {
+        checkRange(len);
+
         System.arraycopy(buf, pos, b, off, len);
 
         pos += len;
@@ -66,9 +69,16 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public int skipBytes(int n) throws IOException {
-        pos += n;
+        if (n < 0)
+            throw new IllegalArgumentException();
+
+        assert pos <= buf.length;
+
+        int toSkip = Math.min(buf.length - pos, n);
 
-        return n;
+        pos += toSkip;
+
+        return toSkip;
     }
 
     /** {@inheritDoc} */
@@ -78,6 +88,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public byte readByte() throws IOException {
+        checkRange(1);
+
         byte res = GridUnsafe.getByte(buf, BYTE_ARR_OFF + pos);
 
         pos += 1;
@@ -92,6 +104,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public short readShort() throws IOException {
+        checkRange(2);
+
         short res = GridUnsafe.getShort(buf, BYTE_ARR_OFF + pos);
 
         pos += 2;
@@ -106,6 +120,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public char readChar() throws IOException {
+        checkRange(2);
+
         char res = GridUnsafe.getChar(buf, BYTE_ARR_OFF + pos);
 
         pos += 2;
@@ -115,6 +131,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public int readInt() throws IOException {
+        checkRange(4);
+
         int res = GridUnsafe.getInt(buf, BYTE_ARR_OFF + pos);
 
         pos += 4;
@@ -124,6 +142,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public long readLong() throws IOException {
+        checkRange(8);
+
         long res = GridUnsafe.getLong(buf, BYTE_ARR_OFF + pos);
 
         pos += 8;
@@ -133,6 +153,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public float readFloat() throws IOException {
+        checkRange(4);
+
         float res = GridUnsafe.getFloat(buf, BYTE_ARR_OFF + pos);
 
         pos += 4;
@@ -142,6 +164,8 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
     /** {@inheritDoc} */
     @Override public double readDouble() throws IOException {
+        checkRange(8);
+
         double res = GridUnsafe.getDouble(buf, BYTE_ARR_OFF + pos);
 
         pos += 8;
@@ -193,4 +217,17 @@ public class HadoopDirectDataInput extends InputStream implements DataInput {
 
         return new String(bytes, StandardCharsets.UTF_8);
     }
+
+    /**
+     * Ensures the position is still within the buffer.
+     *
+     * @throws EOFException if an attempt is made to read beyond the buffer end.
+     */
+    private void checkRange(int bytesToRead) throws EOFException {
+        assert bytesToRead > 0;
+
+        if (pos + bytesToRead - 1 >= buf.length)
+            throw new EOFException("Attempt to read beyond the end of buffer: " + (pos + bytesToRead - 1)
+                + " >= " + buf.length);
+    }
 }


[44/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
index aa2d368..f6ac905 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageIO.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.cache.database.tree.io;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
+import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl;
 import org.apache.ignite.internal.processors.cache.database.MetadataStorage;
@@ -36,10 +38,10 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  * 1. IO should not have any `public static` methods.
  *    We have versioned IOs and any static method will mean that it have to always work in backward
  *    compatible way between all the IO versions. The base class {@link PageIO} has
- *    static methods (like {@code {@link #getPageId(ByteBuffer)}}) intentionally:
+ *    static methods (like {@code {@link #getPageId(long)}}) intentionally:
  *    this base format can not be changed between versions.
  *
- * 2. IO must correctly override {@link #initNewPage(ByteBuffer, long)} method and call super.
+ * 2. IO must correctly override {@link #initNewPage(long, long, int)} method and call super.
  *    We have logic that relies on this behavior.
  *
  * 3. Page IO type ID constant must be declared in this class to have a list of all the
@@ -58,7 +60,7 @@ import org.apache.ignite.internal.processors.cache.database.tree.util.PageLockLi
  *
  * 7. It is almost always preferable to read or write (especially write) page contents using
  *    static methods on {@link PageHandler}. To just initialize new page use
- *    {@link PageHandler#initPage(Page, PageLockListener, PageIO, IgniteWriteAheadLogManager)}
+ *    {@link PageHandler#initPage(PageMemory, Page, PageLockListener, PageIO, IgniteWriteAheadLogManager)}
  *    method with needed IO instance.
  */
 public abstract class PageIO {
@@ -164,6 +166,7 @@ public abstract class PageIO {
     }
 
     /**
+     * @param buf Buffer.
      * @return Page type.
      */
     public static int getType(ByteBuffer buf) {
@@ -171,13 +174,21 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page addres.
+     * @return Page type.
+     */
+    public static int getType(long pageAddr) {
+        return PageUtils.getShort(pageAddr, TYPE_OFF) & 0xFFFF;
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param type Type.
      */
-    public static void setType(ByteBuffer buf, int type) {
-        buf.putShort(TYPE_OFF, (short)type);
+    public static void setType(long pageAddr, int type) {
+        PageUtils.putShort(pageAddr, TYPE_OFF, (short)type);
 
-        assert getType(buf) == type;
+        assert getType(pageAddr) == type;
     }
 
     /**
@@ -189,13 +200,21 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @return Version.
+     */
+    public static int getVersion(long pageAddr) {
+        return PageUtils.getShort(pageAddr, VER_OFF) & 0xFFFF;
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param ver Version.
      */
-    public static void setVersion(ByteBuffer buf, int ver) {
-        buf.putShort(VER_OFF, (short)ver);
+    private static void setVersion(long pageAddr, int ver) {
+        PageUtils.putShort(pageAddr, VER_OFF, (short)ver);
 
-        assert getVersion(buf) == ver;
+        assert getVersion(pageAddr) == ver;
     }
 
     /**
@@ -207,13 +226,37 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     * @return Page ID.
+     */
+    public static long getPageId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, PAGE_ID_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param pageId Page ID.
      */
-    public static void setPageId(ByteBuffer buf, long pageId) {
-        buf.putLong(PAGE_ID_OFF, pageId);
+    public static void setPageId(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, PAGE_ID_OFF, pageId);
+
+        assert getPageId(pageAddr) == pageId;
+    }
+
+    /**
+     * @param pageAddr Page address.
+     * @return Checksum.
+     */
+    public static int getCrc(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CRC_OFF);
+    }
 
-        assert getPageId(buf) == pageId;
+    /**
+     * @param pageAddr Page address.
+     * @param crc Checksum.
+     */
+    public static void setCrc(long pageAddr, int crc) {
+        PageUtils.putInt(pageAddr, CRC_OFF, crc);
     }
 
     /**
@@ -272,18 +315,19 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Page ID.
+     * @param pageSize Page size.
      */
-    public void initNewPage(ByteBuffer buf, long pageId) {
-        setType(buf, getType());
-        setVersion(buf, getVersion());
-        setPageId(buf, pageId);
-        setCrc(buf, 0);
-
-        buf.putLong(RESERVED_1_OFF, 0L);
-        buf.putLong(RESERVED_2_OFF, 0L);
-        buf.putLong(RESERVED_3_OFF, 0L);
+    public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        setType(pageAddr, getType());
+        setVersion(pageAddr, getVersion());
+        setPageId(pageAddr, pageId);
+        setCrc(pageAddr, 0);
+
+        PageUtils.putLong(pageAddr, RESERVED_1_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_2_OFF, 0L);
+        PageUtils.putLong(pageAddr, RESERVED_3_OFF, 0L);
     }
 
     /** {@inheritDoc} */
@@ -292,13 +336,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends PageIO> Q getPageIO(ByteBuffer buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends PageIO> Q getPageIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getPageIO(type, ver);
     }
@@ -339,13 +383,13 @@ public abstract class PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO for either inner or leaf B+Tree page.
      * @throws IgniteCheckedException If failed.
      */
-    public static <Q extends BPlusIO<?>> Q getBPlusIO(ByteBuffer buf) throws IgniteCheckedException {
-        int type = getType(buf);
-        int ver = getVersion(buf);
+    public static <Q extends BPlusIO<?>> Q getBPlusIO(long pageAddr) throws IgniteCheckedException {
+        int type = getType(pageAddr);
+        int ver = getVersion(pageAddr);
 
         return getBPlusIO(type, ver);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
index 8d12f7c..e768f47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PageMetaIO.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -72,117 +73,117 @@ public class PageMetaIO extends PageIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setTreeRoot(buf, 0);
-        setReuseListRoot(buf, 0);
-        setLastSuccessfulFullSnapshotId(buf, 0);
-        setLastSuccessfulSnapshotId(buf, 0);
-        setNextSnapshotTag(buf, 1);
-        setLastSuccessfulSnapshotTag(buf, 0);
-        setLastAllocatedIndex(buf, 0);
-        setCandidateAllocatedIndex(buf, 0);
+        setTreeRoot(pageAddr, 0);
+        setReuseListRoot(pageAddr, 0);
+        setLastSuccessfulFullSnapshotId(pageAddr, 0);
+        setLastSuccessfulSnapshotId(pageAddr, 0);
+        setNextSnapshotTag(pageAddr, 1);
+        setLastSuccessfulSnapshotTag(pageAddr, 0);
+        setLastAllocatedIndex(pageAddr, 0);
+        setCandidateAllocatedIndex(pageAddr, 0);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Tree root page.
      */
-    public long getTreeRoot(ByteBuffer buf) {
-        return buf.getLong(TREE_ROOT_OFF);
+    public long getTreeRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, TREE_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param treeRoot Tree root
      */
-    public void setTreeRoot(@NotNull ByteBuffer buf, long treeRoot) {
-        buf.putLong(TREE_ROOT_OFF, treeRoot);
+    public void setTreeRoot(long pageAddr, long treeRoot) {
+        PageUtils.putLong(pageAddr, TREE_ROOT_OFF, treeRoot);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Reuse list root page.
      */
-    public long getReuseListRoot(ByteBuffer buf) {
-        return buf.getLong(REUSE_LIST_ROOT_OFF);
+    public long getReuseListRoot(long pageAddr) {
+        return PageUtils.getLong(pageAddr, REUSE_LIST_ROOT_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param pageId Root page ID.
      */
-    public void setReuseListRoot(@NotNull ByteBuffer buf, long pageId) {
-        buf.putLong(REUSE_LIST_ROOT_OFF, pageId);
+    public void setReuseListRoot(long pageAddr, long pageId) {
+        PageUtils.putLong(pageAddr, REUSE_LIST_ROOT_OFF, pageId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotId Last successful snapshot id.
      */
-    public void setLastSuccessfulSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
+    public void setLastSuccessfulSnapshotId(long pageAddr, long lastSuccessfulSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF, lastSuccessfulSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulFullSnapshotId Last successful full snapshot id.
      */
-    public void setLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf, long lastSuccessfulFullSnapshotId) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
+    public void setLastSuccessfulFullSnapshotId(long pageAddr, long lastSuccessfulFullSnapshotId) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF, lastSuccessfulFullSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulFullSnapshotId(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
+    public long getLastSuccessfulFullSnapshotId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param nextSnapshotId Next snapshot id.
      */
-    public void setNextSnapshotTag(@NotNull ByteBuffer buf, long nextSnapshotId) {
-        buf.putLong(NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
+    public void setNextSnapshotTag(long pageAddr, long nextSnapshotId) {
+        PageUtils.putLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF, nextSnapshotId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
+    public long getLastSuccessfulSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastSuccessfulSnapshotTag Last successful snapshot tag.
      */
-    public void setLastSuccessfulSnapshotTag(@NotNull ByteBuffer buf, long lastSuccessfulSnapshotTag) {
-        buf.putLong(LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
+    public void setLastSuccessfulSnapshotTag(long pageAddr, long lastSuccessfulSnapshotTag) {
+        PageUtils.putLong(pageAddr, LAST_SUCCESSFUL_FULL_SNAPSHOT_TAG_OFF, lastSuccessfulSnapshotTag);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public long getNextSnapshotTag(@NotNull ByteBuffer buf) {
-        return buf.getLong(NEXT_SNAPSHOT_TAG_OFF);
+    public long getNextSnapshotTag(long pageAddr) {
+        return PageUtils.getLong(pageAddr, NEXT_SNAPSHOT_TAG_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param lastAllocatedIdx Last allocated index.
      */
-    public void setLastAllocatedIndex(@NotNull ByteBuffer buf, int lastAllocatedIdx) {
-        buf.putInt(LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
+    public void setLastAllocatedIndex(long pageAddr, int lastAllocatedIdx) {
+        PageUtils.putInt(pageAddr, LAST_ALLOCATED_INDEX_OFF, lastAllocatedIdx);
     }
 
     /**
@@ -193,17 +194,24 @@ public class PageMetaIO extends PageIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
+     */
+    public int getLastAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, LAST_ALLOCATED_INDEX_OFF);
+    }
+
+    /**
+     * @param pageAddr Page address.
      * @param previousAllocatedIdx Last allocated index.
      */
-    public void setCandidateAllocatedIndex(@NotNull ByteBuffer buf, int previousAllocatedIdx) {
-        buf.putInt(CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
+    public void setCandidateAllocatedIndex(long pageAddr, int previousAllocatedIdx) {
+        PageUtils.putInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF, previousAllocatedIdx);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public int getCandidateAllocatedIndex(@NotNull ByteBuffer buf) {
-        return buf.getInt(CANDIDATE_ALLOCATED_INDEX_OFF);
+    public int getCandidateAllocatedIndex(long pageAddr) {
+        return PageUtils.getInt(pageAddr, CANDIDATE_ALLOCATED_INDEX_OFF);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
index c01d774..aca0725 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/PagePartitionMetaIO.java
@@ -18,8 +18,11 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
-import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagemem.PageUtils;
 
+/**
+ *
+ */
 public class PagePartitionMetaIO extends PageMetaIO {
     /** */
     private static final int SIZE_OFF = PageMetaIO.END_OF_PAGE_META;
@@ -39,13 +42,13 @@ public class PagePartitionMetaIO extends PageMetaIO {
     );
 
     /** {@inheritDoc} */
-    @Override public void initNewPage(ByteBuffer buf, long pageId) {
-        super.initNewPage(buf, pageId);
+    @Override public void initNewPage(long pageAddr, long pageId, int pageSize) {
+        super.initNewPage(pageAddr, pageId, pageSize);
 
-        setSize(buf, 0);
-        setUpdateCounter(buf, 0);
-        setGlobalRemoveId(buf, 0);
-        setPartitionState(buf, (byte)-1);
+        setSize(pageAddr, 0);
+        setUpdateCounter(pageAddr, 0);
+        setGlobalRemoveId(pageAddr, 0);
+        setPartitionState(pageAddr, (byte)-1);
     }
 
     /**
@@ -56,65 +59,65 @@ public class PagePartitionMetaIO extends PageMetaIO {
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition size.
      */
-    public long getSize(ByteBuffer buf) {
-        return buf.getLong(SIZE_OFF);
+    public long getSize(long pageAddr) {
+        return PageUtils.getLong(pageAddr, SIZE_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param size Partition size.
      */
-    public void setSize(ByteBuffer buf, long size) {
-        buf.putLong(SIZE_OFF, size);
+    public void setSize(long pageAddr, long size) {
+        PageUtils.putLong(pageAddr, SIZE_OFF, size);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Partition update counter.
      */
-    public long getUpdateCounter(ByteBuffer buf) {
-        return buf.getLong(UPDATE_CNTR_OFF);
+    public long getUpdateCounter(long pageAddr) {
+        return PageUtils.getLong(pageAddr, UPDATE_CNTR_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param cntr Partition update counter.
      */
-    public void setUpdateCounter(ByteBuffer buf, long cntr) {
-        buf.putLong(UPDATE_CNTR_OFF, cntr);
+    public void setUpdateCounter(long pageAddr, long cntr) {
+        PageUtils.putLong(pageAddr, UPDATE_CNTR_OFF, cntr);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Global remove ID.
      */
-    public long getGlobalRemoveId(ByteBuffer buf) {
-        return buf.getLong(GLOBAL_RMV_ID_OFF);
+    public long getGlobalRemoveId(long pageAddr) {
+        return PageUtils.getLong(pageAddr, GLOBAL_RMV_ID_OFF);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param rmvId Global remove ID.
      */
-    public void setGlobalRemoveId(ByteBuffer buf, long rmvId) {
-        buf.putLong(GLOBAL_RMV_ID_OFF, rmvId);
+    public void setGlobalRemoveId(long pageAddr, long rmvId) {
+        PageUtils.putLong(pageAddr, GLOBAL_RMV_ID_OFF, rmvId);
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public byte getPartitionState(ByteBuffer buf) {
-        return buf.get(PARTITION_STATE_OFF);
+    public byte getPartitionState(long pageAddr) {
+        return PageUtils.getByte(pageAddr, PARTITION_STATE_OFF);
     }
 
     /**
-     * @param buf
-     * @param state
+     * @param pageAddr Page address
+     * @param state State.
      */
-    public void setPartitionState(ByteBuffer buf, byte state) {
-        buf.put(PARTITION_STATE_OFF, state);
+    public void setPartitionState(long pageAddr, byte state) {
+        PageUtils.putByte(pageAddr, PARTITION_STATE_OFF, state);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
index d2813c9..ce8266a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/reuse/ReuseListImpl.java
@@ -65,7 +65,7 @@ public class ReuseListImpl extends PagesList implements ReuseList {
 
     /** {@inheritDoc} */
     @Override public void addForRecycle(ReuseBag bag) throws IgniteCheckedException {
-        put(bag, null, null, 0);
+        put(bag, null, 0, 0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
index fadf74d..97b5a04 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageHandler.java
@@ -20,11 +20,11 @@ package org.apache.ignite.internal.processors.cache.database.tree.util;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.Page;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 import org.apache.ignite.internal.pagemem.wal.record.delta.InitNewPageRecord;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.GridUnsafe;
-import sun.nio.ch.DirectBuffer;
 
 import static java.lang.Boolean.FALSE;
 import static java.lang.Boolean.TRUE;
@@ -35,7 +35,7 @@ import static java.lang.Boolean.TRUE;
 public abstract class PageHandler<X, R> {
     /** */
     private static final PageHandler<Void, Boolean> NOOP = new PageHandler<Void, Boolean>() {
-        @Override public Boolean run(Page page, PageIO io, ByteBuffer buf, Void arg, int intArg)
+        @Override public Boolean run(Page page, PageIO io, long pageAddr, Void arg, int intArg)
             throws IgniteCheckedException {
             return TRUE;
         }
@@ -44,13 +44,13 @@ public abstract class PageHandler<X, R> {
     /**
      * @param page Page.
      * @param io IO.
-     * @param buf Page buffer.
+     * @param pageAddr Page address.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @return Result.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract R run(Page page, PageIO io, ByteBuffer buf, X arg, int intArg)
+    public abstract R run(Page page, PageIO io, long pageAddr, X arg, int intArg)
         throws IgniteCheckedException;
 
     /**
@@ -65,6 +65,7 @@ public abstract class PageHandler<X, R> {
 
     /**
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -74,29 +75,31 @@ public abstract class PageHandler<X, R> {
      */
     public static <X, R> R readPage(
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = readLock(page, lockListener);
+        long pageAddr = readLock(page, lockLsnr);
 
-        if (buf == null)
+        if (pageAddr == 0L)
             return lockFailed;
 
         try {
-            PageIO io = PageIO.getPageIO(buf);
+            PageIO io = PageIO.getPageIO(pageAddr);
 
-            return h.run(page, io, buf, arg, intArg);
+            return h.run(page, io, pageAddr, arg, intArg);
         }
         finally {
-            readUnlock(page, buf, lockListener);
+            readUnlock(page, pageAddr, lockLsnr);
         }
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
@@ -105,92 +108,97 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         X arg,
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        return writePage(page, lockListener, h, null, null, arg, intArg, lockFailed);
+        return writePage(pageMem, page, lockLsnr, h, null, null, arg, intArg, lockFailed);
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param wal WAL manager.
      * @throws IgniteCheckedException If failed.
      */
     public static void initPage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        Boolean res = writePage(page, lockListener, NOOP, init, wal, null, 0, FALSE);
+        Boolean res = writePage(pageMem, page, lockLsnr, NOOP, init, wal, null, 0, FALSE);
 
         assert res == TRUE : res; // It must be newly allocated page, can't be recycled.
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @param lockLsnr Lock listener.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
      */
-    public static ByteBuffer readLock(Page page, PageLockListener lockListener) {
-        lockListener.onBeforeReadLock(page);
+    public static long readLock(Page page, PageLockListener lockLsnr) {
+        lockLsnr.onBeforeReadLock(page);
 
-        ByteBuffer buf = page.getForRead();
+        long pageAddr = page.getForReadPointer();
 
-        lockListener.onReadLock(page, buf);
+        lockLsnr.onReadLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
      * @param page Page.
-     * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param pageAddr Page address.
+     * @param lockLsnr Lock listener.
      */
-    public static void readUnlock(Page page, ByteBuffer buf, PageLockListener lockListener) {
-        lockListener.onReadUnlock(page, buf);
+    public static void readUnlock(Page page, long pageAddr, PageLockListener lockLsnr) {
+        lockLsnr.onReadUnlock(page, pageAddr);
 
         page.releaseRead();
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param tryLock Only try to lock without waiting.
-     * @return Byte buffer or {@code null} if failed to lock due to recycling.
+     * @return Page address or {@code 0} if failed to lock due to recycling.
      */
-    public static ByteBuffer writeLock(Page page, PageLockListener lockListener, boolean tryLock) {
-        lockListener.onBeforeWriteLock(page);
+    public static long writeLock(Page page, PageLockListener lockLsnr, boolean tryLock) {
+        lockLsnr.onBeforeWriteLock(page);
 
-        ByteBuffer buf = tryLock ? page.tryGetForWrite() : page.getForWrite();
+        long pageAddr = tryLock ? page.tryGetForWritePointer() : page.getForWritePointer();
 
-        lockListener.onWriteLock(page, buf);
+        lockLsnr.onWriteLock(page, pageAddr);
 
-        return buf;
+        return pageAddr;
     }
 
     /**
      * @param page Page.
      * @param buf Page buffer.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param dirty Page is dirty.
      */
-    public static void writeUnlock(Page page, ByteBuffer buf, PageLockListener lockListener, boolean dirty) {
-        lockListener.onWriteUnlock(page, buf);
+    public static void writeUnlock(Page page, long buf, PageLockListener lockLsnr, boolean dirty) {
+        lockLsnr.onWriteUnlock(page, buf);
 
         page.releaseWrite(dirty);
     }
 
     /**
      * @param page Page.
-     * @param lockListener Lock listener.
+     * @param lockLsnr Lock listener.
      * @param h Handler.
      * @param init IO for new page initialization or {@code null} if it is an existing page.
+     * @param wal WAL manager.
      * @param arg Argument.
      * @param intArg Argument of type {@code int}.
      * @param lockFailed Result in case of lock failure due to page recycling.
@@ -198,8 +206,9 @@ public abstract class PageHandler<X, R> {
      * @throws IgniteCheckedException If failed.
      */
     public static <X, R> R writePage(
+        PageMemory pageMem,
         Page page,
-        PageLockListener lockListener,
+        PageLockListener lockLsnr,
         PageHandler<X, R> h,
         PageIO init,
         IgniteWriteAheadLogManager wal,
@@ -207,9 +216,9 @@ public abstract class PageHandler<X, R> {
         int intArg,
         R lockFailed
     ) throws IgniteCheckedException {
-        ByteBuffer buf = writeLock(page, lockListener, false);
+        long pageAddr = writeLock(page, lockLsnr, false);
 
-        if (buf == null)
+        if (pageAddr == 0L)
             return lockFailed;
 
         R res;
@@ -218,42 +227,44 @@ public abstract class PageHandler<X, R> {
 
         try {
             if (init != null) // It is a new page and we have to initialize it.
-                doInitPage(page, buf, init, wal);
+                doInitPage(pageMem, page, pageAddr, init, wal);
             else
-                init = PageIO.getPageIO(buf);
+                init = PageIO.getPageIO(pageAddr);
 
-            res = h.run(page, init, buf, arg, intArg);
+            res = h.run(page, init, pageAddr, arg, intArg);
 
             ok = true;
         }
         finally {
-            assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+            assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
             if (h.releaseAfterWrite(page, arg, intArg))
-                writeUnlock(page, buf, lockListener, ok);
+                writeUnlock(page, pageAddr, lockLsnr, ok);
         }
 
         return res;
     }
 
     /**
+     * @param pageMem Page memory.
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param init Initial IO.
      * @param wal Write ahead log.
      * @throws IgniteCheckedException If failed.
      */
     private static void doInitPage(
+        PageMemory pageMem,
         Page page,
-        ByteBuffer buf,
+        long pageAddr,
         PageIO init,
         IgniteWriteAheadLogManager wal
     ) throws IgniteCheckedException {
-        assert PageIO.getCrc(buf) == 0; //TODO GG-11480
+        assert PageIO.getCrc(pageAddr) == 0; //TODO GG-11480
 
         long pageId = page.id();
 
-        init.initNewPage(buf, pageId);
+        init.initNewPage(pageAddr, pageId, pageMem.pageSize());
 
         // Here we should never write full page, because it is known to be new.
         page.fullPageWalRecordPolicy(FALSE);
@@ -288,8 +299,8 @@ public abstract class PageHandler<X, R> {
         long srcArrOff = src.hasArray() ? src.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
         long dstArrOff = dst.hasArray() ? dst.arrayOffset() + GridUnsafe.BYTE_ARR_OFF : 0;
 
-        long srcPtr = src.isDirect() ? ((DirectBuffer)src).address() : 0;
-        long dstPtr = dst.isDirect() ? ((DirectBuffer)dst).address() : 0;
+        long srcPtr = src.isDirect() ? GridUnsafe.bufferAddress(src) : 0;
+        long dstPtr = dst.isDirect() ? GridUnsafe.bufferAddress(dst) : 0;
 
         GridUnsafe.copyMemory(srcArr, srcPtr + srcArrOff + srcOff, dstArr, dstPtr + dstArrOff + dstOff, cnt);
     }
@@ -302,11 +313,31 @@ public abstract class PageHandler<X, R> {
      */
     public static void zeroMemory(ByteBuffer buf, int off, int len) {
         if (buf.isDirect())
-            GridUnsafe.setMemory(((DirectBuffer)buf).address() + off, len, (byte)0);
+            GridUnsafe.setMemory(GridUnsafe.bufferAddress(buf) + off, len, (byte)0);
 
         else {
             for (int i = off; i < off + len; i++)
                 buf.put(i, (byte)0); //TODO Optimize!
         }
     }
+
+    /**
+     * @param srcAddr Source.
+     * @param dstAddr Destination.
+     * @param srcOff Source offset in bytes.
+     * @param dstOff Destination offset in bytes.
+     * @param cnt Bytes count to copy.
+     */
+    public static void copyMemory(long srcAddr, long dstAddr, long srcOff, long dstOff, long cnt) {
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, null, dstAddr + dstOff, cnt);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param len Length.
+     */
+    public static void zeroMemory(long addr, int off, int len) {
+        GridUnsafe.setMemory(addr + off, len, (byte)0);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
index 216bdd3..07cdd43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/util/PageLockListener.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.database.tree.util;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.internal.pagemem.Page;
 
 /**
@@ -31,15 +30,15 @@ public interface PageLockListener {
 
     /**
      * @param page Page.
-     * @param buf Buffer or {@code null} if attempt to lock failed.
+     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
      */
-    public void onWriteLock(Page page, ByteBuffer buf);
+    public void onWriteLock(Page page, long pageAddr);
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void onWriteUnlock(Page page, ByteBuffer buf);
+    public void onWriteUnlock(Page page, long pageAddr);
 
     /**
      * @param page Page.
@@ -48,13 +47,13 @@ public interface PageLockListener {
 
     /**
      * @param page Page.
-     * @param buf Buffer or {@code null} if attempt to lock failed.
+     * @param pageAddr Page address or {@code 0} if attempt to lock failed.
      */
-    public void onReadLock(Page page, ByteBuffer buf);
+    public void onReadLock(Page page, long pageAddr);
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public void onReadUnlock(Page page, ByteBuffer buf);
+    public void onReadUnlock(Page page, long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
index 20d6e90..5fa1daa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateFuture.java
@@ -163,7 +163,8 @@ class GridDhtAtomicSingleUpdateFuture extends GridDhtAtomicAbstractUpdateFuture
 
     /** {@inheritDoc} */
     @Override protected void addFailedKeys(GridNearAtomicUpdateResponse updateRes, Throwable err) {
-        updateRes.addFailedKey(key, err);
+        if (key != null)
+            updateRes.addFailedKey(key, err);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
index 2e38733..b5b11db 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateResponse.java
@@ -341,6 +341,9 @@ public class GridNearAtomicUpdateResponse extends GridCacheMessage implements Gr
      * @param e Error cause.
      */
     public synchronized void addFailedKey(KeyCacheObject key, Throwable e) {
+        assert key != null;
+        assert e != null;
+
         if (failedKeys == null)
             failedKeys = new ConcurrentLinkedQueue<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 783ab96..1926f01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util;
 
 import java.lang.reflect.Field;
+import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
@@ -25,6 +26,7 @@ import java.security.PrivilegedExceptionAction;
 
 import org.apache.ignite.IgniteSystemProperties;
 import sun.misc.Unsafe;
+import sun.nio.ch.DirectBuffer;
 
 /**
  * <p>Wrapper for {@link sun.misc.Unsafe} class.</p>
@@ -1602,4 +1604,14 @@ public abstract class GridUnsafe {
             UNSAFE.putByte(addr, (byte)(val));
         }
     }
+
+    /**
+     * @param buf Direct buffer.
+     * @return Buffer memory address.
+     */
+    public static long bufferAddress(ByteBuffer buf) {
+        assert buf instanceof DirectBuffer : buf;
+
+        return ((DirectBuffer)buf).address();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
index 0c08cd9..8dcd205 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteTree.java
@@ -17,10 +17,8 @@
 
 package org.apache.ignite.internal.util;
 
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.lang.*;
-
-import java.util.*;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.util.lang.GridCursor;
 
 /**
  * Interface for ignite internal tree.
@@ -29,10 +27,11 @@ public interface IgniteTree<L, T> {
     /**
      * Put value in this tree.
      *
-     * @param value value to be associated with the specified key
-     * @return the previous value associated with key
+     * @param val Value to be associated with the specified key.
+     * @return The previous value associated with key.
+     * @throws IgniteCheckedException If failed.
      */
-    T put(T value) throws IgniteCheckedException;
+    public T put(T val) throws IgniteCheckedException;
 
     /**
      * Returns the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the
@@ -40,9 +39,10 @@ public interface IgniteTree<L, T> {
      *
      * @param key the key whose associated value is to be returned
      * @return the value to which the specified key is mapped, or {@code null} if this tree contains no mapping for the
-     * key
+     *  key.
+     * @throws IgniteCheckedException If failed.
      */
-    T findOne(L key) throws IgniteCheckedException;
+    public T findOne(L key) throws IgniteCheckedException;
 
     /**
      * Returns a cursor from lower to upper bounds inclusive.
@@ -50,21 +50,24 @@ public interface IgniteTree<L, T> {
      * @param lower Lower bound or {@code null} if unbounded.
      * @param upper Upper bound or {@code null} if unbounded.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
-    GridCursor<T> find(L lower, L upper) throws IgniteCheckedException;
+    public GridCursor<T> find(L lower, L upper) throws IgniteCheckedException;
 
     /**
      * Removes the mapping for a key from this tree if it is present.
      *
-     * @param key key whose mapping is to be removed from the tree
-     * @return the previous value associated with key, or null if there was no mapping for key.
+     * @param key Key whose mapping is to be removed from the tree.
+     * @return The previous value associated with key, or null if there was no mapping for key.
+     * @throws IgniteCheckedException If failed.
      */
-    T remove(L key) throws IgniteCheckedException;
+    public T remove(L key) throws IgniteCheckedException;
 
     /**
      * Returns the number of elements in this tree.
      *
      * @return the number of elements in this tree
+     * @throws IgniteCheckedException If failed.
      */
-    long size() throws IgniteCheckedException;
+    public long size() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
index ef8b4b4..0052de9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinityNoCacheSelfTest.java
@@ -267,6 +267,11 @@ public class GridAffinityNoCacheSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
index ae81616..ec62c9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoLoadSelfTest.java
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.pagemem.impl;
 
 import java.io.File;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
 import org.apache.ignite.internal.mem.file.MappedFileMemoryProvider;
@@ -32,6 +30,7 @@ import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -190,7 +189,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that initial pages are accessible.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -210,17 +209,17 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are inaccessible using old IDs.
             for (FullPageId id : old) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long pageAddr = page.getForWritePointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0L) {
                         page.releaseWrite(false);
 
                         fail("Was able to acquire page write lock.");
                     }
 
-                    buf = page.getForRead();
+                    pageAddr = page.getForReadPointer();
 
-                    if (buf != null) {
+                    if (pageAddr != 0) {
                         page.releaseRead();
 
                         fail("Was able to acquire page read lock.");
@@ -231,7 +230,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
             // Check that updated pages are accessible using new IDs.
             for (FullPageId id : updated) {
                 try (Page page = mem.page(id.cacheId(), id.pageId())) {
-                    ByteBuffer buf = page.getForWrite();
+                    long buf = page.getForWritePointer();
 
                     assertNotNull(buf);
 
@@ -242,7 +241,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
                         page.releaseWrite(false);
                     }
 
-                    buf = page.getForRead();
+                    buf = page.getForReadPointer();
 
                     assertNotNull(buf);
 
@@ -274,7 +273,7 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), memDir, true,
             sizes);
 
-        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log(), provider, null, PAGE_SIZE, true);
     }
 
     /**
@@ -282,13 +281,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
      * @param val Value to write.
      */
     private void writePage(Page page, int val) {
-        ByteBuffer bytes = page.getForWrite();
+        long bytes = page.getForWritePointer();
 
         try {
             PageIO.setPageId(bytes, page.id());
 
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++)
-                bytes.put(i, (byte)val);
+                PageUtils.putByte(bytes, i, (byte)val);
         }
         finally {
             page.releaseWrite(true);
@@ -302,13 +301,13 @@ public class PageMemoryNoLoadSelfTest extends GridCommonAbstractTest {
     private void readPage(Page page, int expVal) {
         expVal &= 0xFF;
 
-        ByteBuffer bytes = page.getForRead();
+        long pageAddr = page.getForReadPointer();
 
-        assertNotNull(bytes);
+        assert(pageAddr != 0);
 
         try {
             for (int i = PageIO.COMMON_HEADER_END; i < PAGE_SIZE; i++) {
-                int val = bytes.get(i) & 0xFF;
+                int val = PageUtils.getByte(pageAddr, i) & 0xFF;
 
                 assertEquals("Unexpected value at position: " + i, expVal, val);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
index 0224a88..32503d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteIncompleteCacheObjectSelfTest.java
@@ -122,6 +122,11 @@ public class IgniteIncompleteCacheObjectSelfTest extends GridCommonAbstractTest
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
index cb8ba1f..e2767bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/database/tree/io/TrackingPageIOTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.database.tree.io;
 
 import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -25,6 +26,7 @@ import java.util.Set;
 import java.util.TreeSet;
 import java.util.concurrent.ThreadLocalRandom;
 import junit.framework.TestCase;
+import org.apache.ignite.internal.util.GridUnsafe;
 
 /**
  *
@@ -33,13 +35,15 @@ public class TrackingPageIOTest extends TestCase {
     /** Page size. */
     public static final int PAGE_SIZE = 2048;
 
+    /** */
     private final TrackingPageIO io = TrackingPageIO.VERSIONS.latest();
 
     /**
      *
      */
     public void testBasics() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         io.markChanged(buf, 2, 0, -1, PAGE_SIZE);
 
@@ -54,7 +58,8 @@ public class TrackingPageIOTest extends TestCase {
      *
      */
     public void testMarkingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         int cntOfPageToTrack = io.countOfPageToTrack(PAGE_SIZE);
 
@@ -66,7 +71,8 @@ public class TrackingPageIOTest extends TestCase {
      *
      */
     public void testZeroingRandomly() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         for (int i = 0; i < 1001; i++)
             checkMarkingRandomly(buf, i, true);
@@ -87,7 +93,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         Map<Long, Boolean> map = new HashMap<>();
 
@@ -122,8 +128,12 @@ public class TrackingPageIOTest extends TestCase {
         }
     }
 
+    /**
+     * @throws Exception If failed.
+     */
     public void testFindNextChangedPage() throws Exception {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         for (int i = 0; i < 101; i++)
             checkFindingRandomly(buf, i);
@@ -144,7 +154,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         try {
             TreeSet<Long> setIdx = new TreeSet<>();
@@ -174,8 +184,12 @@ public class TrackingPageIOTest extends TestCase {
         }
     }
 
+    /**
+     *
+     */
     public void testMerging() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
@@ -185,7 +199,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         TreeSet<Long> setIdx = new TreeSet<>();
 
@@ -209,8 +223,12 @@ public class TrackingPageIOTest extends TestCase {
             assertFalse(io.wasChanged(buf, i, 5, 4, PAGE_SIZE));
     }
 
+    /**
+     *
+     */
     public void testMerging_MarksShouldBeDropForSuccessfulBackup() {
-        ByteBuffer buf = ByteBuffer.allocate(PAGE_SIZE);
+        ByteBuffer buf = ByteBuffer.allocateDirect(PAGE_SIZE);
+        buf.order(ByteOrder.nativeOrder());
 
         ThreadLocalRandom rand = ThreadLocalRandom.current();
 
@@ -220,7 +238,7 @@ public class TrackingPageIOTest extends TestCase {
 
         assert basePageId >= 0;
 
-        PageIO.setPageId(buf, basePageId);
+        PageIO.setPageId(GridUnsafe.bufferAddress(buf), basePageId);
 
         TreeSet<Long> setIdx = new TreeSet<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
index cfc58b0..5f38bb0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeReuseSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.database;
 
-import java.nio.ByteBuffer;
 import java.util.HashSet;
 import java.util.Set;
 import org.apache.ignite.IgniteCheckedException;
@@ -47,6 +46,9 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         assertTrue(TestReuseList.checkNoLocks());
     }
 
+    /**
+     *
+     */
     private static class TestReuseList extends ReuseListImpl {
         /** */
         private static ThreadLocal<Set<Long>> readLocks = new ThreadLocal<Set<Long>>() {
@@ -88,15 +90,15 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadLock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
             assertTrue(readLocks.get().add(page.id()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadUnlock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
             assertTrue(readLocks.get().remove(page.id()));
         }
@@ -107,18 +109,18 @@ public class BPlusTreeReuseSelfTest extends BPlusTreeSelfTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf == null)
+        @Override public void onWriteLock(Page page, long pageAddr) {
+            if (pageAddr == 0L)
                 return; // Failed to lock.
 
-            checkPageId(page, buf);
+            checkPageId(page, pageAddr);
 
             assertTrue(writeLocks.get().add(page.id()));
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
+        @Override public void onWriteUnlock(Page page, long pageAddr) {
+            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
 
             assertTrue(writeLocks.get().remove(page.id()));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
index 4dcb7a9..928c86d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/BPlusTreeSelfTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.database.DataStructure;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
@@ -1165,10 +1166,10 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    public static void checkPageId(Page page, ByteBuffer buf) {
-        long pageId = PageIO.getPageId(buf);
+    public static void checkPageId(Page page, long pageAddr) {
+        long pageId = PageIO.getPageId(pageAddr);
 
         // Page ID must be 0L for newly allocated page, for reused page effective ID must remain the same.
         if (pageId != 0L && page.id() != pageId)
@@ -1232,18 +1233,18 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override protected int compare(BPlusIO<Long> io, ByteBuffer buf, int idx, Long n2)
+        @Override protected int compare(BPlusIO<Long> io, long pageAddr, int idx, Long n2)
             throws IgniteCheckedException {
-            Long n1 = io.getLookupRow(this, buf, idx);
+            Long n1 = io.getLookupRow(this, pageAddr, idx);
 
             return Long.compare(n1, n2);
         }
 
         /** {@inheritDoc} */
-        @Override protected Long getRow(BPlusIO<Long> io, ByteBuffer buf, int idx) throws IgniteCheckedException {
+        @Override protected Long getRow(BPlusIO<Long> io, long pageAddr, int idx) throws IgniteCheckedException {
             assert io.canGetRow() : io;
 
-            return io.getLookupRow(this, buf, idx);
+            return io.getLookupRow(this, pageAddr, idx);
         }
 
         /**
@@ -1280,11 +1281,11 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
-                long pageId = PageIO.getPageId(buf);
+        @Override public void onReadLock(Page page, long pageAddr) {
+            if (pageAddr != 0L) {
+                long pageId = PageIO.getPageId(pageAddr);
 
-                checkPageId(page, buf);
+                checkPageId(page, pageAddr);
 
                 assertNull(locks(true).put(page.id(), pageId));
             }
@@ -1293,10 +1294,10 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onReadUnlock(Page page, ByteBuffer buf) {
-            checkPageId(page, buf);
+        @Override public void onReadUnlock(Page page, long pageAddr) {
+            checkPageId(page, pageAddr);
 
-            long pageId = PageIO.getPageId(buf);
+            long pageId = PageIO.getPageId(pageAddr);
 
             assertEquals(Long.valueOf(pageId), locks(true).remove(page.id()));
         }
@@ -1307,11 +1308,11 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteLock(Page page, ByteBuffer buf) {
-            if (buf != null) {
-                checkPageId(page, buf);
+        @Override public void onWriteLock(Page page, long pageAddr) {
+            if (pageAddr != 0L) {
+                checkPageId(page, pageAddr);
 
-                long pageId = PageIO.getPageId(buf);
+                long pageId = PageIO.getPageId(pageAddr);
 
                 if (pageId == 0L)
                     pageId = page.id(); // It is a newly allocated page.
@@ -1323,8 +1324,8 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void onWriteUnlock(Page page, ByteBuffer buf) {
-            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(buf)));
+        @Override public void onWriteUnlock(Page page, long pageAddr) {
+            assertEquals(effectivePageId(page.id()), effectivePageId(PageIO.getPageId(pageAddr)));
 
             assertEquals(Long.valueOf(page.id()), locks(false).remove(page.id()));
         }
@@ -1391,15 +1392,15 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long buf, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(buf, pageSize);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx)
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx)
             throws IgniteCheckedException {
             Long row = srcIo.getLookupRow(null, src, srcIdx);
 
@@ -1415,16 +1416,21 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
+        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(long pageAddr, int off, Long row) {
             checkNotRemoved(row);
 
-            buf.putLong(off, row);
+            PageUtils.putLong(pageAddr, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            Long row = buf.getLong(offset(idx));
+            Long row = PageUtils.getLong(pageAddr, offset(idx));
 
             checkNotRemoved(row);
 
@@ -1441,7 +1447,7 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, PAGE_SIZE, true);
 
         pageMem.start();
 
@@ -1461,34 +1467,39 @@ public class BPlusTreeSelfTest extends GridCommonAbstractTest {
     private static final class LongLeafIO extends BPlusLeafIO<Long> {
         /**
          */
-        protected LongLeafIO() {
+        LongLeafIO() {
             super(LONG_LEAF_IO, 1, 8);
         }
 
         /** {@inheritDoc} */
-        @Override public int getMaxCount(ByteBuffer buf) {
+        @Override public int getMaxCount(long pageAddr, int pageSize) {
             if (MAX_PER_PAGE != 0)
                 return MAX_PER_PAGE;
 
-            return super.getMaxCount(buf);
+            return super.getMaxCount(pageAddr, pageSize);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) throws IgniteCheckedException {
+            throw new UnsupportedOperationException();
         }
 
         /** {@inheritDoc} */
-        @Override public void storeByOffset(ByteBuffer buf, int off, Long row) {
-            buf.putLong(off, row);
+        @Override public void storeByOffset(long pageAddr, int off, Long row) {
+            PageUtils.putLong(pageAddr, off, row);
         }
 
         /** {@inheritDoc} */
-        @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<Long> srcIo, ByteBuffer src, int srcIdx) {
+        @Override public void store(long dst, int dstIdx, BPlusIO<Long> srcIo, long src, int srcIdx) {
             assert srcIo == this;
 
-            dst.putLong(offset(dstIdx), src.getLong(offset(srcIdx)));
+            PageUtils.putLong(dst, offset(dstIdx), PageUtils.getLong(src, offset(srcIdx)));
         }
 
         /** {@inheritDoc} */
-        @Override public Long getLookupRow(BPlusTree<Long,?> tree, ByteBuffer buf, int idx)
+        @Override public Long getLookupRow(BPlusTree<Long,?> tree, long pageAddr, int idx)
             throws IgniteCheckedException {
-            return buf.getLong(offset(idx));
+            return PageUtils.getLong(pageAddr, offset(idx));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
index 2b93d48..d0d495e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/FreeListImplSelfTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.mem.unsafe.UnsafeMemoryProvider;
 import org.apache.ignite.internal.pagemem.PageIdAllocator;
 import org.apache.ignite.internal.pagemem.PageMemory;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.pagemem.impl.PageMemoryNoStoreImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
@@ -315,7 +316,7 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < sizes.length; i++)
             sizes[i] = 1024 * MB / CPUS;
 
-        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize);
+        PageMemory pageMem = new PageMemoryNoStoreImpl(log, new UnsafeMemoryProvider(sizes), null, pageSize, true);
 
         pageMem.start();
 
@@ -395,6 +396,11 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         @Override public void link(long link) {
             this.link = link;
         }
+
+        /** {@inheritDoc} */
+        @Override public int hash() {
+            throw new UnsupportedOperationException();
+        }
     }
 
     /**
@@ -458,6 +464,13 @@ public class FreeListImplSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            PageUtils.putBytes(addr, 0, data);
+
+            return data.length;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(ByteBuffer buf, int off, int len) throws IgniteCheckedException {
             buf.put(data, off, len);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
index 252ea82..7eeda4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/database/MetadataStorageSelfTest.java
@@ -160,6 +160,6 @@ public class MetadataStorageSelfTest extends GridCommonAbstractTest {
 
         DirectMemoryProvider provider = new MappedFileMemoryProvider(log(), allocationPath, clean, sizes);
 
-        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE);
+        return new PageMemoryNoStoreImpl(log, provider, null, PAGE_SIZE, true);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
index cc71813..de3111d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/H2Cursor.java
@@ -31,13 +31,13 @@ import org.h2.result.*;
  */
 public class H2Cursor implements Cursor {
     /** */
-    final GridCursor<GridH2Row> cursor;
+    private final GridCursor<GridH2Row> cursor;
 
     /** */
-    final IgniteBiPredicate<Object,Object> filter;
+    private final IgniteBiPredicate<Object,Object> filter;
 
     /** */
-    final long time = U.currentTimeMillis();
+    private final long time = U.currentTimeMillis();
 
     /**
      * @param cursor Cursor.
@@ -50,6 +50,9 @@ public class H2Cursor implements Cursor {
         this.filter = filter;
     }
 
+    /**
+     * @param cursor Cursor.
+     */
     public H2Cursor(GridCursor<GridH2Row> cursor) {
         this(cursor, null);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
index 0593c60..5c60b16 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2Tree.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database;
 
-import java.nio.ByteBuffer;
 import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.pagemem.PageMemory;
@@ -76,9 +75,9 @@ public abstract class H2Tree extends BPlusTree<SearchRow, GridH2Row> {
     }
 
     /** {@inheritDoc} */
-    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, ByteBuffer buf, int idx)
+    @Override protected GridH2Row getRow(BPlusIO<SearchRow> io, long pageAddr, int idx)
         throws IgniteCheckedException {
-        return (GridH2Row)io.getLookupRow(this, buf, idx);
+        return (GridH2Row)io.getLookupRow(this, pageAddr, idx);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 19cbbf9..6cabd77 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.query.h2.database;
 
-import java.nio.ByteBuffer;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -87,9 +86,9 @@ public class H2TreeIndex extends GridH2IndexBase {
             tree = new H2Tree(name, cctx.offheap().reuseListForIndex(name), cctx.cacheId(),
                 dbMgr.pageMemory(), cctx.shared().wal(), cctx.offheap().globalRemoveId(),
                 tbl.rowFactory(), page.pageId().pageId(), page.isAllocated()) {
-                @Override protected int compare(BPlusIO<SearchRow> io, ByteBuffer buf, int idx, SearchRow row)
+                @Override protected int compare(BPlusIO<SearchRow> io, long pageAddr, int idx, SearchRow row)
                     throws IgniteCheckedException {
-                    return compareRows(getRow(io, buf, idx), row);
+                    return compareRows(getRow(io, pageAddr, idx), row);
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
index 6868205..f697121 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2InnerIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.database.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
@@ -53,22 +54,31 @@ public class H2InnerIO extends BPlusInnerIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
+        GridH2Row row0 = (GridH2Row)row;
+
+        assert row0.link != 0;
+
+        PageUtils.putLong(pageAddr, off, row0.link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long pageAddr, int idx)
         throws IgniteCheckedException {
-        long link = getLink(buf, idx);
+        long link = getLink(pageAddr, idx);
 
         return ((H2Tree)tree).getRowFactory().getRow(link);
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
-        long link = ((H2RowLinkIO)srcIo).getLink(src, srcIdx);
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<SearchRow> srcIo, long srcPageAddr, int srcIdx) {
+        long link = ((H2RowLinkIO)srcIo).getLink(srcPageAddr, srcIdx);
 
-        dst.putLong(offset(dstIdx), link);
+        PageUtils.putLong(dstPageAddr, offset(dstIdx), link);
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
index ce517a8..26cbdc5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2LeafIO.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.query.h2.database.io;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageUtils;
 import org.apache.ignite.internal.processors.cache.database.tree.BPlusTree;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusLeafIO;
@@ -53,22 +54,31 @@ public class H2LeafIO extends BPlusLeafIO<SearchRow> implements H2RowLinkIO {
     }
 
     /** {@inheritDoc} */
-    @Override public void store(ByteBuffer dst, int dstIdx, BPlusIO<SearchRow> srcIo, ByteBuffer src, int srcIdx) {
+    @Override public void storeByOffset(long pageAddr, int off, SearchRow row) {
+        GridH2Row row0 = (GridH2Row)row;
+
+        assert row0.link != 0;
+
+        PageUtils.putLong(pageAddr, off, row0.link);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void store(long dstPageAddr, int dstIdx, BPlusIO<SearchRow> srcIo, long srcPageAddr, int srcIdx) {
         assert srcIo == this;
 
-        dst.putLong(offset(dstIdx), getLink(src, srcIdx));
+        PageUtils.putLong(dstPageAddr, offset(dstIdx), getLink(srcPageAddr, srcIdx));
     }
 
     /** {@inheritDoc} */
-    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, ByteBuffer buf, int idx)
+    @Override public SearchRow getLookupRow(BPlusTree<SearchRow,?> tree, long pageAddr, int idx)
         throws IgniteCheckedException {
-        long link = getLink(buf, idx);
+        long link = getLink(pageAddr, idx);
 
         return ((H2Tree)tree).getRowFactory().getRow(link);
     }
 
     /** {@inheritDoc} */
-    @Override public long getLink(ByteBuffer buf, int idx) {
-        return buf.getLong(offset(idx));
+    @Override public long getLink(long pageAddr, int idx) {
+        return PageUtils.getLong(pageAddr, offset(idx));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
index 9c4737d..ce69197 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/io/H2RowLinkIO.java
@@ -17,16 +17,14 @@
 
 package org.apache.ignite.internal.processors.query.h2.database.io;
 
-import java.nio.ByteBuffer;
-
 /**
  * Row link IO.
  */
 public interface H2RowLinkIO {
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Row link.
      */
-    public long getLink(ByteBuffer buf, int idx);
+    public long getLink(long pageAddr, int idx);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index ac48fcb..bdd43ef 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -164,4 +164,9 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer, Cac
     @Override public Value[] getValueList() {
         throw new UnsupportedOperationException();
     }
+
+    /** {@inheritDoc} */
+    @Override public int hash() {
+        throw new UnsupportedOperationException();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
index e9f41c8..81201c8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/IgniteSqlDistributedJoinSelfTest.java
@@ -17,18 +17,18 @@
 
 package org.apache.ignite.internal.processors.query;
 
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.query.*;
-import org.apache.ignite.cache.query.annotations.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.plugin.*;
-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.junits.common.*;
-
-import java.util.*;
+import java.util.List;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
  * Tests for correct distributed sql joins.
@@ -36,8 +36,14 @@ import java.util.*;
 public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
     private static final int NODES_COUNT = 2;
+
+    /** */
     private static final int ORG_COUNT = NODES_COUNT;
+
+    /** */
     private static final int PERSON_PER_ORG_COUNT = 50;
 
     /** {@inheritDoc} */
@@ -81,7 +87,7 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testNonCollocatedDistributedJoin() throws Exception {
         CacheConfiguration ccfg1 = cacheConfig("pers", true, String.class, Person.class);
@@ -141,11 +147,19 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /**
+     *
+     */
     private static class Person {
+        /** */
         @QuerySqlField(index = true)
         private String id;
+
+        /** */
         @QuerySqlField(index = true)
         private String orgId;
+
+        /** */
         @QuerySqlField(index = true)
         private String name;
 
@@ -162,9 +176,15 @@ public class IgniteSqlDistributedJoinSelfTest extends GridCommonAbstractTest {
         public void setName(String name) { this.name = name; }
     }
 
+    /**
+     *
+     */
     private static class Organization {
+        /** */
         @QuerySqlField(index = true)
         private String id;
+
+        /** */
         @QuerySqlField(index = true)
         private String name;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 593ba95..b9152fa 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -694,6 +694,11 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         }
 
         /** {@inheritDoc} */
+        @Override public int putValue(long addr) throws IgniteCheckedException {
+            return 0;
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
             throws IgniteCheckedException {
             return false;


[16/50] [abbrv] ignite git commit: IGNITE-4493: ODBC: Added missing diagnostic records for error cases

Posted by sb...@apache.org.
IGNITE-4493: ODBC: Added missing diagnostic records for error cases

This closes #1396


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 9e8e9798dae8b92cfbf4d940856dec35d3e30f54
Parents: bf118aa
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Jan 10 11:10:56 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 11:10:56 2017 +0300

----------------------------------------------------------------------
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++++++++++-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 ++-
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++++++++++----
 modules/platforms/cpp/odbc/src/connection.cpp   |   2 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 ++++
 modules/platforms/cpp/odbc/src/odbc.cpp         | 118 ++++---------
 modules/platforms/cpp/odbc/src/statement.cpp    | 176 ++++++++++++++++---
 7 files changed, 484 insertions(+), 135 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
index 13a5ea6..0b6df93 100644
--- a/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/api_robustness_test.cpp
@@ -50,7 +50,7 @@ using ignite::impl::binary::BinaryUtils;
 /**
  * Test setup fixture.
  */
-struct ApiRobustnessTestSuiteFixture 
+struct ApiRobustnessTestSuiteFixture
 {
     void Prepare()
     {
@@ -206,6 +206,33 @@ struct ApiRobustnessTestSuiteFixture
 
         // Operation is not supported. However, there should be no crash.
         BOOST_CHECK(ret == SQL_ERROR);
+
+        CheckSQLStatementDiagnosticError("HY106");
+    }
+
+    void CheckSQLDiagnosticError(int16_t handleType, SQLHANDLE handle, const std::string& expectSqlState)
+    {
+        SQLCHAR state[ODBC_BUFFER_SIZE];
+        SQLINTEGER nativeError = 0;
+        SQLCHAR message[ODBC_BUFFER_SIZE];
+        SQLSMALLINT messageLen = 0;
+
+        SQLRETURN ret = SQLGetDiagRec(handleType, handle, 1, state, &nativeError, message, sizeof(message), &messageLen);
+
+        const std::string sqlState = reinterpret_cast<char*>(state);
+        BOOST_REQUIRE_EQUAL(ret, SQL_SUCCESS);
+        BOOST_REQUIRE_EQUAL(sqlState, expectSqlState);
+        BOOST_REQUIRE(messageLen > 0);
+    }
+
+    void CheckSQLStatementDiagnosticError(const std::string& expectSqlState)
+    {
+        CheckSQLDiagnosticError(SQL_HANDLE_STMT, stmt, expectSqlState);
+    }
+
+    void CheckSQLConnectionDiagnosticError(const std::string& expectSqlState)
+    {
+        CheckSQLDiagnosticError(SQL_HANDLE_DBC, dbc, expectSqlState);
     }
 
     /**
@@ -234,6 +261,43 @@ struct ApiRobustnessTestSuiteFixture
     SQLHSTMT stmt;
 };
 
+SQLSMALLINT unsupportedC[] = {
+        SQL_C_INTERVAL_YEAR,
+        SQL_C_INTERVAL_MONTH,
+        SQL_C_INTERVAL_DAY,
+        SQL_C_INTERVAL_HOUR,
+        SQL_C_INTERVAL_MINUTE,
+        SQL_C_INTERVAL_SECOND,
+        SQL_C_INTERVAL_YEAR_TO_MONTH,
+        SQL_C_INTERVAL_DAY_TO_HOUR,
+        SQL_C_INTERVAL_DAY_TO_MINUTE,
+        SQL_C_INTERVAL_DAY_TO_SECOND,
+        SQL_C_INTERVAL_HOUR_TO_MINUTE,
+        SQL_C_INTERVAL_HOUR_TO_SECOND,
+        SQL_C_INTERVAL_MINUTE_TO_SECOND
+    };
+
+SQLSMALLINT unsupportedSql[] = {
+        SQL_WVARCHAR,
+        SQL_WLONGVARCHAR,
+        SQL_REAL,
+        SQL_NUMERIC,
+        SQL_TYPE_TIME,
+        SQL_INTERVAL_MONTH,
+        SQL_INTERVAL_YEAR,
+        SQL_INTERVAL_YEAR_TO_MONTH,
+        SQL_INTERVAL_DAY,
+        SQL_INTERVAL_HOUR,
+        SQL_INTERVAL_MINUTE,
+        SQL_INTERVAL_SECOND,
+        SQL_INTERVAL_DAY_TO_HOUR,
+        SQL_INTERVAL_DAY_TO_MINUTE,
+        SQL_INTERVAL_DAY_TO_SECOND,
+        SQL_INTERVAL_HOUR_TO_MINUTE,
+        SQL_INTERVAL_HOUR_TO_SECOND,
+        SQL_INTERVAL_MINUTE_TO_SECOND
+    };
+
 BOOST_FIXTURE_TEST_SUITE(ApiRobustnessTestSuite, ApiRobustnessTestSuiteFixture)
 
 BOOST_AUTO_TEST_CASE(TestSQLDriverConnect)
@@ -516,6 +580,19 @@ BOOST_AUTO_TEST_CASE(TestSQLBindCol)
 
     ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt);
 
+    //Unsupported data types
+    for(int i = 0; i < sizeof(unsupportedC)/sizeof(unsupportedC[0]); ++i)
+    {
+        ret = SQLBindCol(stmt, 1, unsupportedC[i], &ind1, sizeof(ind1), &len1);
+        BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+        CheckSQLStatementDiagnosticError("HY003");
+    }
+
+    // Size is negative.
+    ret = SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, -1, &len1);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLStatementDiagnosticError("HY090");
+
     // Size is null.
     SQLBindCol(stmt, 1, SQL_C_SLONG, &ind1, 0, &len1);
 
@@ -545,6 +622,24 @@ BOOST_AUTO_TEST_CASE(TestSQLBindParameter)
 
     ODBC_FAIL_ON_ERROR(ret, SQL_HANDLE_STMT, stmt);
 
+    //Unsupported parameter type : output
+    SQLBindParameter(stmt, 2, SQL_PARAM_OUTPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1);
+    CheckSQLStatementDiagnosticError("HY105");
+
+    //Unsupported parameter type : input/output
+    SQLBindParameter(stmt, 2, SQL_PARAM_INPUT_OUTPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, sizeof(ind1), &len1);
+    CheckSQLStatementDiagnosticError("HY105");
+
+
+    //Unsupported data types
+    for(int i = 0; i < sizeof(unsupportedSql)/sizeof(unsupportedSql[0]); ++i)
+    {
+        ret = SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, unsupportedSql[i], 100, 100, &ind1, sizeof(ind1), &len1);
+        BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+        CheckSQLStatementDiagnosticError("HYC00");
+    }
+
+
     // Size is null.
     SQLBindParameter(stmt, 2, SQL_PARAM_INPUT, SQL_C_SLONG, SQL_INTEGER, 100, 100, &ind1, 0, &len1);
 
@@ -1111,4 +1206,20 @@ BOOST_AUTO_TEST_CASE(TestSQLError)
     SQLError(0, 0, 0, 0, 0, 0, 0, 0);
 }
 
+BOOST_AUTO_TEST_CASE(TestSQLDiagnosticRecords)
+{
+    Connect("DRIVER={Apache Ignite};address=127.0.0.1:11110;cache=cache");
+
+    SQLHANDLE hnd;
+    SQLRETURN ret;
+
+    ret = SQLAllocHandle(SQL_HANDLE_DESC, dbc, &hnd);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLConnectionDiagnosticError("IM001");
+
+    ret = SQLFreeStmt(stmt, 4);
+    BOOST_REQUIRE_EQUAL(ret, SQL_ERROR);
+    CheckSQLStatementDiagnosticError("HY092");
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
index 517fe4e..5d8901d 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/common_types.h
@@ -113,15 +113,37 @@ namespace ignite
              */
             SQL_STATE_HY001_MEMORY_ALLOCATION,
 
+            /**
+             * The argument TargetType was neither a valid data type
+             * nor SQL_C_DEFAULT
+             */
+            SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE,
+
             /** Invalid use of null pointer. */
             SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER,
 
             /** Function sequence error. */
             SQL_STATE_HY010_SEQUENCE_ERROR,
 
+            /**
+             * Invalid string or buffer length
+             */
+            SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH,
+
+            /**
+             * Option type was out of range.
+             */
+            SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE,
+
             /** Column type out of range. */
             SQL_STATE_HY097_COLUMN_TYPE_OUT_OF_RANGE,
 
+            /** The value specified for the argument InputOutputType was invalid. */
+            SQL_STATE_HY105_INVALID_PARAMETER_TYPE,
+
+            /** The value specified for the argument FetchOrientation was invalid. */
+            SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE,
+
             /**
              * The driver does not support the feature of ODBC behavior that
              * the application requested.
@@ -132,7 +154,12 @@ namespace ignite
              * The connection timeout period expired before the data source
              * responded to the request.
              */
-            SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT
+            SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT,
+
+            /**
+             * Driver does not support this function.
+             */
+            SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED
         };
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
index db56660..596fc66 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/statement.h
@@ -57,24 +57,15 @@ namespace ignite
             ~Statement();
 
             /**
-             * Bind result column to specified data buffer.
+             * Bind result column to data buffer provided by application
              *
              * @param columnIdx Column index.
-             * @param buffer Buffer to put column data to.
+             * @param targetType Type of target buffer.
+             * @param targetValue Pointer to target buffer.
+             * @param bufferLength Length of target buffer.
+             * @param strLengthOrIndicator Pointer to the length/indicator buffer.
              */
-            void BindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer);
-
-            /**
-             * Unbind specified column buffer.
-             *
-             * @param columnIdx Column index.
-             */
-            void UnbindColumn(uint16_t columnIdx);
-
-            /**
-             * Unbind all column buffers.
-             */
-            void UnbindAllColumns();
+            void BindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator);
 
             /**
              * Set column binding offset pointer.
@@ -101,21 +92,17 @@ namespace ignite
              * Bind parameter.
              *
              * @param paramIdx Parameter index.
-             * @param param Parameter.
-             */
-            void BindParameter(uint16_t paramIdx, const app::Parameter& param);
-
-            /**
-             * Unbind specified parameter.
-             *
-             * @param paramIdx Parameter index.
+             * @param ioType Type of the parameter (input/output).
+             * @param bufferType The data type of the parameter.
+             * @param paramSqlType The SQL data type of the parameter.
+             * @param columnSize  The size of the column or expression of the corresponding parameter marker.
+             * @param decDigits  The decimal digits of the column or expression of the corresponding parameter marker.
+             * @param buffer A pointer to a buffer for the parameter's data.
+             * @param bufferLen Length of the ParameterValuePtr buffer in bytes.
+             * @param resLen A pointer to a buffer for the parameter's length.
              */
-            void UnbindParameter(uint16_t paramIdx);
-
-            /**
-             * Unbind all parameters.
-             */
-            void UnbindAllParameters();
+            void BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                               SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen);
 
             /**
              * Set statement attribute.
@@ -171,7 +158,7 @@ namespace ignite
              * @param query SQL query.
              */
             void PrepareSqlQuery(const std::string& query);
-            
+
             /**
              * Execute SQL query.
              *
@@ -254,11 +241,24 @@ namespace ignite
             void ExecuteGetTypeInfoQuery(int16_t sqlType);
 
             /**
+             * Free resources
+             * @param option indicates what needs to be freed
+             */
+            void FreeResources(int16_t option);
+
+            /**
              * Close statement.
              */
             void Close();
 
             /**
+             * Fetch query result row with offset
+             * @param orientation Fetch type
+             * @param offset Fetch offset
+             */
+            void FetchScroll(int16_t orientation, int64_t offset);
+
+            /**
              * Fetch query result row.
              */
             void FetchRow();
@@ -362,14 +362,75 @@ namespace ignite
         private:
             IGNITE_NO_COPY_ASSIGNMENT(Statement);
 
+
+            /**
+             * Bind result column to specified data buffer.
+             *
+             * @param columnIdx Column index.
+             * @param buffer Buffer to put column data to.
+             */
+            void SafeBindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer);
+
+            /**
+             * Unbind specified column buffer.
+             *
+             * @param columnIdx Column index.
+             */
+            void SafeUnbindColumn(uint16_t columnIdx);
+
+            /**
+             * Unbind all column buffers.
+             */
+            void SafeUnbindAllColumns();
+
+            /**
+             * Bind result column to data buffer provided by application
+             *
+             * @param columnIdx Column index.
+             * @param targetType Type of target buffer.
+             * @param targetValue Pointer to target buffer.
+             * @param bufferLength Length of target buffer.
+             * @param strLengthOrIndicator Pointer to the length/indicator buffer.
+             * @return Operation result.
+             */
+            SqlResult InternalBindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator);
+
             /**
              * Bind parameter.
              *
              * @param paramIdx Parameter index.
              * @param param Parameter.
+             */
+            void SafeBindParameter(uint16_t paramIdx, const app::Parameter& param);
+
+            /**
+             * Unbind specified parameter.
+             *
+             * @param paramIdx Parameter index.
+             */
+            void SafeUnbindParameter(uint16_t paramIdx);
+
+            /**
+             * Unbind all parameters.
+             */
+            void SafeUnbindAllParameters();
+
+            /**
+             * Bind parameter.
+             *
+             * @param paramIdx Parameter index.
+             * @param ioType Type of the parameter (input/output).
+             * @param bufferType The data type of the parameter.
+             * @param paramSqlType The SQL data type of the parameter.
+             * @param columnSize  The size of the column or expression of the corresponding parameter marker.
+             * @param decDigits  The decimal digits of the column or expression of the corresponding parameter marker.
+             * @param buffer A pointer to a buffer for the parameter's data.
+             * @param bufferLen Length of the ParameterValuePtr buffer in bytes.
+             * @param resLen A pointer to a buffer for the parameter's length.
              * @return Operation result.
              */
-            SqlResult InternalBindParameter(uint16_t paramIdx, const app::Parameter& param);
+            SqlResult InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                            SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen);
 
             /**
              * Set statement attribute.
@@ -403,6 +464,14 @@ namespace ignite
              */
             SqlResult InternalGetColumnData(uint16_t columnIdx, app::ApplicationDataBuffer& buffer);
 
+
+            /**
+             * Free resources
+             * @param option indicates what needs to be freed
+             * @return Operation result.
+             */
+            SqlResult InternalFreeResources(int16_t option);
+
             /**
              * Close statement.
              * Internal call.
@@ -418,7 +487,7 @@ namespace ignite
              * @return Operation result.
              */
             SqlResult InternalPrepareSqlQuery(const std::string& query);
-            
+
             /**
              * Execute SQL query.
              *
@@ -435,6 +504,14 @@ namespace ignite
             SqlResult InternalExecuteSqlQuery();
 
             /**
+             * Fetch query result row with offset
+             * @param orientation Fetch type
+             * @param offset Fetch offset
+             * @return Operation result.
+             */
+            SqlResult InternalFetchScroll(int16_t orientation, int64_t offset);
+
+            /**
              * Fetch query result row.
              *
              * @return Operation result.
@@ -621,7 +698,7 @@ namespace ignite
 
             /** Offset added to pointers to change binding of parameters. */
             int* paramBindOffset;
-            
+
             /** Offset added to pointers to change binding of column data. */
             int* columnBindOffset;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 2afa42d..4a6de5e 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -226,7 +226,7 @@ namespace ignite
 
             OdbcProtocolHeader hdr;
 
-            int64_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
+            size_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
 
             if (received != sizeof(hdr))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message header");

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
index 215d77f..3c411d2 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnostic_record.cpp
@@ -64,17 +64,35 @@ namespace
     /** SQL state HY001 constant. */
     const std::string STATE_HY001 = "HY001";
 
+    /** SQL state HY003 constant. */
+    const std::string STATE_HY003 = "HY003";
+
     /** SQL state HY009 constant. */
     const std::string STATE_HY009 = "HY009";
 
     /** SQL state HY010 constant. */
     const std::string STATE_HY010 = "HY010";
 
+    /** SQL state HY092 constant. */
+    const std::string STATE_HY092 = "HY092";
+
+    /** SQL state HY105 constant. */
+    const std::string STATE_HY105 = "HY105";
+
+    /** SQL state HY106 constant. */
+    const std::string STATE_HY106 = "HY106";
+
     /** SQL state HYC00 constant. */
     const std::string STATE_HYC00 = "HYC00";
 
     /** SQL state HYT01 constant. */
     const std::string STATE_HYT01 = "HYT01";
+
+    /** SQL state HY090 constant. */
+    const std::string STATE_HY090 = "HY090";
+
+    /** SQL state IM001 constant. */
+    const std::string STATE_IM001 = "IM001";
 }
 
 namespace ignite
@@ -130,6 +148,8 @@ namespace ignite
 
                 if (odbcSubclasses.empty())
                 {
+                    // This is a fixed list taken from ODBC doc.
+                    // Please do not add/remove values here.
                     odbcSubclasses.insert("01S00");
                     odbcSubclasses.insert("01S01");
                     odbcSubclasses.insert("01S02");
@@ -234,18 +254,36 @@ namespace ignite
                     case SQL_STATE_HY001_MEMORY_ALLOCATION:
                         return STATE_HY001;
 
+                    case SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE:
+                        return STATE_HY003;
+
                     case SQL_STATE_HY009_INVALID_USE_OF_NULL_POINTER:
                         return STATE_HY009;
 
                     case SQL_STATE_HY010_SEQUENCE_ERROR:
                         return STATE_HY010;
 
+                    case SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH:
+                        return STATE_HY090;
+
+                    case SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE:
+                        return STATE_HY092;
+
+                    case SQL_STATE_HY105_INVALID_PARAMETER_TYPE:
+                        return STATE_HY105;
+
+                    case SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE:
+                        return STATE_HY106;
+
                     case SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED:
                         return STATE_HYC00;
 
                     case SQL_STATE_HYT01_CONNECTIOIN_TIMEOUT:
                         return STATE_HYT01;
 
+                    case SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED:
+                        return STATE_IM001;
+
                     default:
                         break;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index dbec55f..fd69c0d 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -31,6 +31,7 @@
 #include "ignite/odbc/dsn_config.h"
 #include "ignite/odbc.h"
 
+
 namespace ignite
 {
     SQLRETURN SQLGetInfo(SQLHDBC        conn,
@@ -71,6 +72,23 @@ namespace ignite
                 return SQLAllocStmt(parent, result);
 
             case SQL_HANDLE_DESC:
+            {
+                using odbc::Connection;
+                Connection *connection = reinterpret_cast<Connection*>(parent);
+
+                if (!connection)
+                    return SQL_INVALID_HANDLE;
+
+                if (result)
+                    *result = 0;
+
+                connection->GetDiagnosticRecords().Reset();
+                connection->AddStatusRecord(odbc::SQL_STATE_IM001_FUNCTION_NOT_SUPPORTED,
+                                            "The HandleType argument was SQL_HANDLE_DESC, and "
+                                            "the driver does not support allocating a descriptor handle");
+
+                return SQL_ERROR;
+            }
             default:
                 break;
         }
@@ -199,39 +217,15 @@ namespace ignite
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        switch (option)
+        if (option == SQL_DROP)
         {
-            case SQL_DROP:
-            {
-                delete statement;
-
-                break;
-            }
-
-            case SQL_CLOSE:
-            {
-                return SQLCloseCursor(stmt);
-            }
-
-            case SQL_UNBIND:
-            {
-                statement->UnbindAllColumns();
-
-                break;
-            }
-
-            case SQL_RESET_PARAMS:
-            {
-                statement->UnbindAllParameters();
-
-                break;
-            }
-
-            default:
-                return SQL_ERROR;
+            delete statement;
+            return SQL_SUCCESS;
         }
 
-        return SQL_SUCCESS;
+        statement->FreeResources(option);
+
+        return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
     SQLRETURN SQLCloseCursor(SQLHSTMT stmt)
@@ -419,29 +413,14 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLBindCol called: index=%d, type=%d\n", colNum, targetType);
+        LOG_MSG("SQLBindCol called: index=%d, type=%d targetValue=%p bufferLength=%d\n", colNum, targetType, targetValue, bufferLength);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        IgniteSqlType driverType = ToDriverType(targetType);
-
-        if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
-            return SQL_ERROR;
-
-        if (bufferLength < 0)
-            return SQL_ERROR;
-
-        if (targetValue || strLengthOrIndicator)
-        {
-            ApplicationDataBuffer dataBuffer(driverType, targetValue, bufferLength, strLengthOrIndicator);
-
-            statement->BindColumn(colNum, dataBuffer);
-        }
-        else
-            statement->UnbindColumn(colNum);
+        statement->BindColumn(colNum, targetType, targetValue, bufferLength, strLengthOrIndicator);
 
         return statement->GetDiagnosticRecords().GetReturnCode();
     }
@@ -464,13 +443,19 @@ namespace ignite
 
     SQLRETURN SQLFetchScroll(SQLHSTMT stmt, SQLSMALLINT orientation, SQLLEN offset)
     {
+        using odbc::Statement;
+
         LOG_MSG("SQLFetchScroll called\n");
         LOG_MSG("Orientation: %d, Offset: %d\n", orientation, offset);
 
-        if (orientation != SQL_FETCH_NEXT)
-            return SQL_ERROR;
+        Statement *statement = reinterpret_cast<Statement*>(stmt);
+
+        if (!statement)
+            return SQL_INVALID_HANDLE;
+
+        statement->FetchScroll(orientation, offset);
 
-        return SQLFetch(stmt);
+        return statement->GetDiagnosticRecords().GetReturnCode();
     }
 
     SQLRETURN SQLExtendedFetch(SQLHSTMT         stmt,
@@ -617,12 +602,7 @@ namespace ignite
                                SQLLEN       bufferLen,
                                SQLLEN*      resLen)
     {
-        using namespace odbc::type_traits;
-
         using odbc::Statement;
-        using odbc::app::ApplicationDataBuffer;
-        using odbc::app::Parameter;
-        using odbc::type_traits::IsSqlTypeSupported;
 
         LOG_MSG("SQLBindParameter called: %d, %d, %d\n", paramIdx, bufferType, paramSqlType);
 
@@ -631,27 +611,7 @@ namespace ignite
         if (!statement)
             return SQL_INVALID_HANDLE;
 
-        if (ioType != SQL_PARAM_INPUT)
-            return SQL_ERROR;
-
-        if (!IsSqlTypeSupported(paramSqlType))
-            return SQL_ERROR;
-
-        IgniteSqlType driverType = ToDriverType(bufferType);
-
-        if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
-            return SQL_ERROR;
-
-        if (buffer)
-        {
-            ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen);
-
-            Parameter param(dataBuffer, paramSqlType, columnSize, decDigits);
-
-            statement->BindParameter(paramIdx, param);
-        }
-        else
-            statement->UnbindParameter(paramIdx);
+        statement->BindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen);
 
         return statement->GetDiagnosticRecords().GetReturnCode();
     }
@@ -717,13 +677,13 @@ namespace ignite
     }
 
     SQLRETURN SQLDescribeCol(SQLHSTMT       stmt,
-                             SQLUSMALLINT   columnNum, 
+                             SQLUSMALLINT   columnNum,
                              SQLCHAR*       columnNameBuf,
                              SQLSMALLINT    columnNameBufLen,
                              SQLSMALLINT*   columnNameLen,
-                             SQLSMALLINT*   dataType, 
+                             SQLSMALLINT*   dataType,
                              SQLULEN*       columnSize,
-                             SQLSMALLINT*   decimalDigits, 
+                             SQLSMALLINT*   decimalDigits,
                              SQLSMALLINT*   nullable)
     {
         using odbc::Statement;

http://git-wip-us.apache.org/repos/asf/ignite/blob/9e8e9798/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index 02c6dd9..09ad81c 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -50,26 +50,57 @@ namespace ignite
             // No-op.
         }
 
-        void Statement::BindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer)
+        void Statement::BindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            IGNITE_ODBC_API_CALL(InternalBindColumn(columnIdx, targetType, targetValue, bufferLength, strLengthOrIndicator));
+        }
+
+        SqlResult Statement::InternalBindColumn(uint16_t columnIdx, int16_t targetType, void* targetValue, SqlLen bufferLength, SqlLen* strLengthOrIndicator)
+        {
+            using namespace odbc::type_traits;
+            IgniteSqlType driverType = ToDriverType(targetType);
+
+            if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE, "The argument TargetType was not a valid data type.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (bufferLength < 0)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY090_INVALID_STRING_OR_BUFFER_LENGTH,
+                    "The value specified for the argument BufferLength was less than 0.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (targetValue || strLengthOrIndicator)
+            {
+                app::ApplicationDataBuffer dataBuffer(driverType, targetValue, bufferLength, strLengthOrIndicator);
 
+                SafeBindColumn(columnIdx, dataBuffer);
+            }
+            else
+                SafeUnbindColumn(columnIdx);
+
+            return SQL_RESULT_SUCCESS;
+        }
+
+        void Statement::SafeBindColumn(uint16_t columnIdx, const app::ApplicationDataBuffer& buffer)
+        {
             columnBindings[columnIdx] = buffer;
 
             columnBindings[columnIdx].SetPtrToOffsetPtr(&columnBindOffset);
         }
 
-        void Statement::UnbindColumn(uint16_t columnIdx)
+        void Statement::SafeUnbindColumn(uint16_t columnIdx)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
-
             columnBindings.erase(columnIdx);
         }
 
-        void Statement::UnbindAllColumns()
+        void Statement::SafeUnbindAllColumns()
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
-
             columnBindings.clear();
         }
 
@@ -108,14 +139,21 @@ namespace ignite
             return SQL_RESULT_SUCCESS;
         }
 
-        void Statement::BindParameter(uint16_t paramIdx, const app::Parameter& param)
+        void Statement::BindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                      SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen)
         {
-            IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, param));
+            IGNITE_ODBC_API_CALL(InternalBindParameter(paramIdx, ioType, bufferType, paramSqlType, columnSize, decDigits, buffer, bufferLen, resLen));
         }
 
-
-        SqlResult Statement::InternalBindParameter(uint16_t paramIdx, const app::Parameter& param)
+        SqlResult Statement::InternalBindParameter(uint16_t paramIdx, int16_t ioType, int16_t bufferType, int16_t paramSqlType,
+                                                   SqlUlen columnSize, int16_t decDigits, void* buffer, SqlLen bufferLen, SqlLen* resLen)
         {
+            using namespace odbc::type_traits;
+            using odbc::Statement;
+            using odbc::app::ApplicationDataBuffer;
+            using odbc::app::Parameter;
+            using odbc::type_traits::IsSqlTypeSupported;
+
             if (paramIdx == 0)
             {
                 AddStatusRecord(SQL_STATE_24000_INVALID_CURSOR_STATE,
@@ -124,24 +162,60 @@ namespace ignite
                 return SQL_RESULT_ERROR;
             }
 
-            paramBindings[paramIdx] = param;
+            if (ioType != SQL_PARAM_INPUT)
+            {
+                AddStatusRecord(SQL_STATE_HY105_INVALID_PARAMETER_TYPE,
+                    "The value specified for the argument InputOutputType was not SQL_PARAM_INPUT.");
 
-            paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(&paramBindOffset);
+                return SQL_RESULT_ERROR;
+            }
+
+            if (!IsSqlTypeSupported(paramSqlType))
+            {
+                AddStatusRecord(SQL_STATE_HYC00_OPTIONAL_FEATURE_NOT_IMPLEMENTED,
+                    "Data type is not supported.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            IgniteSqlType driverType = ToDriverType(bufferType);
+
+            if (driverType == IGNITE_ODBC_C_TYPE_UNSUPPORTED)
+            {
+                AddStatusRecord(odbc::SQL_STATE_HY003_INVALID_APPLICATION_BUFFER_TYPE,
+                    "The argument TargetType was not a valid data type.");
+
+                return SQL_RESULT_ERROR;
+            }
+
+            if (buffer)
+            {
+                ApplicationDataBuffer dataBuffer(driverType, buffer, bufferLen, resLen);
+
+                Parameter param(dataBuffer, paramSqlType, columnSize, decDigits);
+
+                SafeBindParameter(paramIdx, param);
+            }
+            else
+                SafeUnbindParameter(paramIdx);
 
             return SQL_RESULT_SUCCESS;
         }
 
-        void Statement::UnbindParameter(uint16_t paramIdx)
+        void Statement::SafeBindParameter(uint16_t paramIdx, const app::Parameter& param)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            paramBindings[paramIdx] = param;
 
-            paramBindings.erase(paramIdx);
+            paramBindings[paramIdx].GetBuffer().SetPtrToOffsetPtr(&paramBindOffset);
         }
 
-        void Statement::UnbindAllParameters()
+        void Statement::SafeUnbindParameter(uint16_t paramIdx)
         {
-            IGNITE_ODBC_API_CALL_ALWAYS_SUCCESS;
+            paramBindings.erase(paramIdx);
+        }
 
+        void Statement::SafeUnbindAllParameters()
+        {
             paramBindings.clear();
         }
 
@@ -536,6 +610,50 @@ namespace ignite
             return currentQuery->Execute();
         }
 
+        void Statement::FreeResources(int16_t option)
+        {
+            IGNITE_ODBC_API_CALL(InternalFreeResources(option));
+        }
+
+        SqlResult Statement::InternalFreeResources(int16_t option)
+        {
+            switch (option)
+            {
+                case SQL_DROP:
+                {
+                    AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, "Deprecated, call SQLFreeHandle instead");
+
+                    return SQL_RESULT_ERROR;
+                }
+
+                case SQL_CLOSE:
+                {
+                    return InternalClose();
+                }
+
+                case SQL_UNBIND:
+                {
+                    SafeUnbindAllColumns();
+
+                    break;
+                }
+
+                case SQL_RESET_PARAMS:
+                {
+                    SafeUnbindAllParameters();
+
+                    break;
+                }
+
+                default:
+                {
+                    AddStatusRecord(SQL_STATE_HY092_OPTION_TYPE_OUT_OF_RANGE, "The value specified for the argument Option was invalid");
+                    return SQL_RESULT_ERROR;
+                }
+            }
+            return SQL_RESULT_SUCCESS;
+        }
+
         void Statement::Close()
         {
             IGNITE_ODBC_API_CALL(InternalClose());
@@ -545,7 +663,7 @@ namespace ignite
         {
             if (!currentQuery.get())
                 return SQL_RESULT_SUCCESS;
-            
+
             SqlResult result = currentQuery->Close();
 
             if (result == SQL_RESULT_SUCCESS)
@@ -554,6 +672,24 @@ namespace ignite
             return result;
         }
 
+        void Statement::FetchScroll(int16_t orientation, int64_t offset)
+        {
+            IGNITE_ODBC_API_CALL(InternalFetchScroll(orientation, offset));
+        }
+
+        SqlResult Statement::InternalFetchScroll(int16_t orientation, int64_t offset)
+        {
+            UNREFERENCED_PARAMETER(offset);
+
+            if (orientation != SQL_FETCH_NEXT)
+            {
+                AddStatusRecord(SQL_STATE_HY106_FETCH_TYPE_OUT_OF_RANGE, "The value specified for the argument FetchOrientation was not SQL_FETCH_NEXT.");
+                return SQL_RESULT_ERROR;
+            }
+
+            return InternalFetchRow();
+        }
+
         void Statement::FetchRow()
         {
             IGNITE_ODBC_API_CALL(InternalFetchRow());


[24/50] [abbrv] ignite git commit: ignite-4293 Do not need store deserialized value for BinaryMarshaller

Posted by sb...@apache.org.
ignite-4293 Do not need store deserialized value for BinaryMarshaller


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

Branch: refs/heads/ignite-gg-11810-1
Commit: d10946b8b107dee499a5798a4679ab5d14499764
Parents: abc8b90
Author: Alexandr Kuramshin <ak...@gridgain.com>
Authored: Fri Jan 13 12:26:39 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jan 13 12:26:39 2017 +0300

----------------------------------------------------------------------
 .../processors/cacheobject/IgniteCacheObjectProcessorImpl.java | 5 ++---
 .../java/org/apache/ignite/cache/store/jdbc/model/Person.java  | 2 +-
 .../processors/cache/CacheEntryProcessorCopySelfTest.java      | 6 ++++--
 .../processors/cache/GridCacheBasicStoreAbstractTest.java      | 2 +-
 .../distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java | 5 +----
 5 files changed, 9 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/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 208ec62..614c612 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
@@ -244,9 +244,8 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
 
         CacheMemoryMode memMode = ccfg.getMemoryMode();
 
-        boolean storeVal = ctx.config().isPeerClassLoadingEnabled() ||
-            GridQueryProcessor.isEnabled(ccfg) ||
-            !ccfg.isCopyOnRead();
+        boolean storeVal = !ccfg.isCopyOnRead() || (!isBinaryEnabled(ccfg) &&
+            (GridQueryProcessor.isEnabled(ccfg) || ctx.config().isPeerClassLoadingEnabled()));
 
         CacheObjectContext res = new CacheObjectContext(ctx,
             ccfg.getName(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
index ddf309b..52ddfc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/model/Person.java
@@ -196,7 +196,7 @@ public class Person implements Serializable {
     @Override public String toString() {
         return "Person [id=" + id +
             ", orgId=" + orgId +
-            ", birthday=" + birthday.getTime() +
+            ", birthday=" + (birthday == null ? null : birthday.getTime()) +
             ", name=" + name +
             "]";
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
index 21395e6..f44889b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheEntryProcessorCopySelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -93,8 +94,9 @@ public class CacheEntryProcessorCopySelfTest extends GridCommonAbstractTest {
             doTest(true, false, OLD_VAL, 1);
 
             // One deserialization due to copyOnRead == true.
-            // Additional deserialization in case p2p enabled due to storeValue == true on update entry.
-            doTest(true, true, NEW_VAL, p2pEnabled ? 2 : 1);
+            // Additional deserialization in case p2p enabled and not BinaryMarshaller due to storeValue == true on update entry.
+            doTest(true, true, NEW_VAL, p2pEnabled &&
+                !(grid.configuration().getMarshaller() instanceof BinaryMarshaller) ? 2 : 1);
 
             // No deserialization.
             doTest(false, false, NEW_VAL, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
index 8ddd737..026b618 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheBasicStoreAbstractTest.java
@@ -571,7 +571,7 @@ public abstract class GridCacheBasicStoreAbstractTest extends GridCommonAbstract
 
             assert cached != null;
 
-            assert cached == val : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']';
+            assert cached.equals(val) : "Cached value mismatch [expected=" + val + ", cached=" + cached + ']';
 
             // Make sure that value is coming from cache, not from store.
             checkLastMethod(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/d10946b8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
index 3f3f84f..e8a6cfb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionsDisabledSelfTest.java
@@ -117,10 +117,7 @@ public class GridCacheDhtEvictionsDisabledSelfTest extends GridCommonAbstractTes
             assertNotNull(v1);
             assertNotNull(v2);
 
-            if (affinity(cache).mapKeyToNode(key).isLocal())
-                assertSame(v1, v2);
-            else
-                assertEquals(v1, v2);
+            assertEquals(v1, v2);
         }
     }
 }
\ No newline at end of file


[07/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-1.8.2'

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-1.8.2'


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 72f03ea7d8bc92de624817655e21f05c268a3399
Parents: 228d97b da5b68c
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 30 15:13:06 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 30 15:13:06 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java | 159 ++++++++++++++++---
 .../communication/tcp/TcpCommunicationSpi.java  |  20 +--
 .../tcp/TcpCommunicationSpiMBean.java           |   5 +-
 ...mmunicationBalancePairedConnectionsTest.java |  28 ++++
 .../IgniteCommunicationBalanceTest.java         |  25 ++-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 ------
 ...micMessageRecoveryPairedConnectionsTest.java |  47 ++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 .../yardstick/cache/IgniteIoTestBenchmark.java  |  73 ---------
 9 files changed, 250 insertions(+), 160 deletions(-)
----------------------------------------------------------------------



[18/50] [abbrv] ignite git commit: IGNITE-4532 .NET: Fix build warnings

Posted by sb...@apache.org.
IGNITE-4532 .NET: Fix build warnings


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 87fbb275eb9d11084485e6b4314b5bf7f8d42a1a
Parents: 825fe9c
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 16:02:26 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 16:02:26 2017 +0300

----------------------------------------------------------------------
 .../platforms/cpp/common/project/vs/common.vcxproj  |  1 +
 modules/platforms/cpp/jni/project/vs/jni.vcxproj    |  1 +
 .../Apache.Ignite.AspNet.Tests.csproj               |  7 ++++++-
 .../Apache.Ignite.AspNet.csproj                     |  5 +++++
 .../Apache.Ignite.Benchmarks.csproj                 |  5 +++++
 .../Apache.Ignite.Core.Tests.TestDll.csproj         |  5 +++++
 .../Apache.Ignite.Core.Tests.csproj                 |  5 +++++
 .../dotnet/Apache.Ignite.Core.Tests/TestUtils.cs    |  1 -
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj    | 16 ++++++++--------
 .../Impl/Unmanaged/UnmanagedUtils.cs                |  4 ++--
 .../Apache.Ignite.EntityFramework.Tests.csproj      |  5 +++++
 .../Apache.Ignite.EntityFramework.csproj            |  9 +++++++--
 .../Impl/ArrayDbDataReader.cs                       |  5 +++++
 .../Impl/DataReaderResult.cs                        |  2 ++
 .../Impl/DbCommandInfo.cs                           | 11 +++++++++++
 .../Impl/DbCommandProxy.cs                          |  1 +
 .../Apache.Ignite.Linq/Apache.Ignite.Linq.csproj    |  5 +++++
 .../Apache.Ignite.Log4Net.csproj                    |  5 +++++
 .../Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs    | 16 ++++++++--------
 .../Apache.Ignite.NLog/Apache.Ignite.NLog.csproj    |  5 +++++
 modules/platforms/dotnet/Apache.Ignite.sln          |  3 +++
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj       |  5 +++++
 .../Apache.Ignite.Examples.csproj                   |  3 +++
 .../Apache.Ignite.ExamplesDll.csproj                |  4 ++++
 .../Services/MapService.cs                          |  2 --
 25 files changed, 107 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/cpp/common/project/vs/common.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/common/project/vs/common.vcxproj b/modules/platforms/cpp/common/project/vs/common.vcxproj
index 99fd551..b4431f8 100644
--- a/modules/platforms/cpp/common/project/vs/common.vcxproj
+++ b/modules/platforms/cpp/common/project/vs/common.vcxproj
@@ -97,6 +97,7 @@
       <Optimization>Disabled</Optimization>
       <PreprocessorDefinitions>_SCL_SECURE_NO_WARNINGS;WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
     </ClCompile>
     <Link>
       <SubSystem>Windows</SubSystem>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/cpp/jni/project/vs/jni.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/jni.vcxproj b/modules/platforms/cpp/jni/project/vs/jni.vcxproj
index f080a68..cb03197 100644
--- a/modules/platforms/cpp/jni/project/vs/jni.vcxproj
+++ b/modules/platforms/cpp/jni/project/vs/jni.vcxproj
@@ -97,6 +97,7 @@
       <Optimization>Disabled</Optimization>
       <PreprocessorDefinitions>WIN32;_DEBUG;_WINDOWS;_USRDLL;IGNITEJVM_EXPORTS;_CRT_SECURE_NO_WARNINGS;IGNITE_IMPL;%(PreprocessorDefinitions)</PreprocessorDefinitions>
       <AdditionalIncludeDirectories>$(JAVA_HOME)\include;$(JAVA_HOME)\include\win32;$(ProjectDir)\..\..\..\common\include;$(ProjectDir)\..\..\..\common\os\win\include;$(ProjectDir)\..\..\include;$(ProjectDir)\..\..\os\win\include;%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
+      <DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
     </ClCompile>
     <Link>
       <SubSystem>Windows</SubSystem>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
index aed74db..628942d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Apache.Ignite.AspNet.Tests.csproj
@@ -21,7 +21,8 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
-    <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -30,6 +31,7 @@
     <DefineConstants>TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.AspNet.Tests.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="nunit.framework">
       <HintPath>..\packages\NUnit.Runners.2.6.3\tools\nunit.framework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
index 2e501c1..4508776 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Apache.Ignite.AspNet.csproj
@@ -28,6 +28,7 @@
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>Apache.Ignite.AspNet.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -37,6 +38,10 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>Apache.Ignite.AspNet.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\Release\Apache.Ignite.AspNet.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
index 811047c..65dfc1e 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/Apache.Ignite.Benchmarks.csproj
@@ -27,12 +27,17 @@
     <DefineConstants>DEBUG</DefineConstants>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
index b33023a..db7a4d1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.TestDll/Apache.Ignite.Core.Tests.TestDll.csproj
@@ -23,12 +23,17 @@
     <OutputPath>bin\Debug\</OutputPath>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 55adfe4..78a08d2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -28,6 +28,7 @@
     <AllowUnsafeBlocks>true</AllowUnsafeBlocks>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -35,6 +36,10 @@
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="log4net, Version=1.2.15.0, Culture=neutral, PublicKeyToken=669e0ddf0bb1aa2a, processorArchitecture=MSIL">

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
index ad27398..8360bf1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestUtils.cs
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-#pragma warning disable S2360 // Optional parameters should not be used
 namespace Apache.Ignite.Core.Tests
 {
     using System;

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index b2db0a7..a80dfc0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -21,6 +21,7 @@
     <DocumentationFile>bin\x64\Debug\Apache.Ignite.Core.XML</DocumentationFile>
     <RunCodeAnalysis>false</RunCodeAnalysis>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x64'">
     <PlatformTarget>x64</PlatformTarget>
@@ -30,6 +31,7 @@
     <Optimize>true</Optimize>
     <DocumentationFile>bin\x64\Release\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Debug|x86'">
     <PlatformTarget>x86</PlatformTarget>
@@ -39,6 +41,7 @@
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\x86\Debug\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|x86'">
     <PlatformTarget>x86</PlatformTarget>
@@ -48,6 +51,7 @@
     <Optimize>true</Optimize>
     <DocumentationFile>bin\x86\Release\Apache.Ignite.Core.XML</DocumentationFile>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -67,6 +71,7 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -76,6 +81,7 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <DebugType>none</DebugType>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />
@@ -514,10 +520,10 @@
   </ItemGroup>
   <ItemGroup Condition="'$(Platform)' == 'AnyCPU'">
     <EmbeddedResource Include="$(ProjectDir)..\..\cpp\jni\project\vs\x64\$(Configuration)\ignite.jni.dll">
-      <Link>resources\$(Configuration)\x64\ignite.jni.dll</Link>
+      <Link>resources\x64\ignite.jni.dll</Link>
     </EmbeddedResource>
     <EmbeddedResource Include="$(ProjectDir)..\..\cpp\jni\project\vs\Win32\$(Configuration)\ignite.jni.dll" Condition="'$(Configuration)' == 'Release' Or Exists('$(ProjectDir)..\..\cpp\jni\project\vs\Win32\$(Configuration)\ignite.jni.dll')">
-      <Link>resources\$(Configuration)\x86\ignite.jni.dll</Link>
+      <Link>resources\x86\ignite.jni.dll</Link>
     </EmbeddedResource>
   </ItemGroup>
   <ItemGroup>
@@ -537,12 +543,6 @@
   <ItemGroup>
     <Folder Include="Impl\Common\JavaObjects\" />
   </ItemGroup>
-  <ItemGroup>
-    <ProjectReference Include="..\..\cpp\jni\project\vs\jni.vcxproj">
-      <Project>{4f7e4917-4612-4b96-9838-025711ade391}</Project>
-      <Name>jni</Name>
-    </ProjectReference>
-  </ItemGroup>
   <Import Project="$(MSBuildToolsPath)\Microsoft.CSharp.targets" />
   <PropertyGroup Condition="'$(Platform)' == 'AnyCPU'">
     <PreBuildEvent>cd "$(ProjectDir)"

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index 0a2a1f0..34baee0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -38,9 +38,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [SuppressMessage("Microsoft.Design", "CA1065:DoNotRaiseExceptionsInUnexpectedLocations")]
         static UnmanagedUtils()
         {
-            var platfrom = Environment.Is64BitProcess ? "x64" : "x86";
+            var platform = Environment.Is64BitProcess ? "x64" : "x86";
 
-            var resName = string.Format("{0}.{1}", platfrom, IgniteUtils.FileIgniteJniDll);
+            var resName = string.Format("{0}.{1}", platform, IgniteUtils.FileIgniteJniDll);
 
             var path = IgniteUtils.UnpackEmbeddedResource(resName, IgniteUtils.FileIgniteJniDll);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
index 9711087..b03d5f0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/Apache.Ignite.EntityFramework.Tests.csproj
@@ -20,6 +20,7 @@
     <DefineConstants>DEBUG;TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -28,6 +29,7 @@
     <DefineConstants>TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -35,6 +37,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.Tests.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="EntityFramework">
       <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
index 7082182..372c2ed 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Apache.Ignite.EntityFramework.csproj
@@ -21,9 +21,10 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
-    <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
     <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
     <DocumentationFile>bin\Debug\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>pdbonly</DebugType>
@@ -32,7 +33,8 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DebugType>none</DebugType>
-	<DocumentationFile>bin\Release\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <DocumentationFile>bin\Release\Apache.Ignite.EntityFramework.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -40,6 +42,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.EntityFramework.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="EntityFramework, Version=6.0.0.0, Culture=neutral, PublicKeyToken=b77a5c561934e089, processorArchitecture=MSIL">
       <HintPath>..\packages\EntityFramework.6.1.3\lib\net40\EntityFramework.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
index 89523f4..0cf31e5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/ArrayDbDataReader.cs
@@ -117,6 +117,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override long GetBytes(int ordinal, long dataOffset, byte[] buffer, int bufferOffset, int length)
         {
             Debug.Assert(buffer != null);
@@ -137,6 +138,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override long GetChars(int ordinal, long dataOffset, char[] buffer, int bufferOffset, int length)
         {
             Debug.Assert(buffer != null);
@@ -193,8 +195,11 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
         public override int GetValues(object[] values)
         {
+            Debug.Assert(values != null);
+
             var row = GetRow();
 
             var size = Math.Min(row.Length, values.Length);

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
index 48f763c..b683184 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DataReaderResult.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.EntityFramework.Impl
     using System.Collections.Generic;
     using System.Data;
     using System.Data.Common;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
 
     /// <summary>
@@ -38,6 +39,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         /// <summary>
         /// Initializes a new instance of the <see cref="DataReaderResult"/> class.
         /// </summary>
+        [SuppressMessage("Microsoft.Usage", "CA2202: Do not call Dispose more than one time on an object")]
         public DataReaderResult(IDataReader reader)
         {
             try

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
index 6b5db05..b80764c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.EntityFramework.Impl
     using System.Data.Entity.Core.Common.CommandTrees;
     using System.Data.Entity.Core.Metadata.Edm;
     using System.Diagnostics;
+    using System.Diagnostics.CodeAnalysis;
     using System.Linq;
 
     /// <summary>
@@ -166,8 +167,18 @@ namespace Apache.Ignite.EntityFramework.Impl
             public static readonly EntitySetComparer Instance = new EntitySetComparer();
 
             /** <inheritdoc /> */
+            [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods")]
             public int Compare(EntitySetBase x, EntitySetBase y)
             {
+                if (x == null && y == null)
+                    return 0;
+
+                if (x == null)
+                    return -1;
+
+                if (y == null)
+                    return 1;
+
                 return string.CompareOrdinal(x.Name, y.Name);
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
index e3353d5..0eb90ae 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandProxy.cs
@@ -73,6 +73,7 @@ namespace Apache.Ignite.EntityFramework.Impl
         }
 
         /** <inheritDoc /> */
+        [SuppressMessage("Microsoft.Security", "CA2100:Review SQL queries for security vulnerabilities", Justification = "This class is just a proxy.")]
         public override string CommandText
         {
             get { return _command.CommandText; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
index 72e050c..04b4916 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Apache.Ignite.Linq.csproj
@@ -21,6 +21,7 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>..\Apache.Ignite.Core\Apache.Ignite.Core.ruleset</CodeAnalysisRuleSet>
     <RunCodeAnalysis>false</RunCodeAnalysis>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
@@ -30,6 +31,7 @@
     <ErrorReport>prompt</ErrorReport>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
     <DocumentationFile>bin\Release\Apache.Ignite.Linq.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.Linq.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Remotion.Linq, Version=2.0.0.0, Culture=neutral, PublicKeyToken=fee00910d6e5f53b, processorArchitecture=MSIL">
       <HintPath>..\packages\Remotion.Linq.2.0.1\lib\net40\Remotion.Linq.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
index af5b7e4..142ce73 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/Apache.Ignite.Log4Net.csproj
@@ -22,6 +22,7 @@
     <WarningLevel>4</WarningLevel>
     <RunCodeAnalysis>true</RunCodeAnalysis>
     <CodeAnalysisRuleSet>AllRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>none</DebugType>
@@ -30,6 +31,7 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DocumentationFile>bin\Release\Apache.Ignite.Log4Net.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -37,6 +39,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.Log4Net.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="log4net, Version=1.2.15.0, Culture=neutral, PublicKeyToken=669e0ddf0bb1aa2a, processorArchitecture=MSIL">
       <HintPath>..\packages\log4net.2.0.5\lib\net40-full\log4net.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
index 058176f..1270600 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Log4Net/IgniteLog4NetLogger.cs
@@ -55,7 +55,7 @@ namespace Apache.Ignite.Log4Net
         /// <summary>
         /// Logs the specified message.
         /// </summary>
-        /// <param name="logLevel">The level.</param>
+        /// <param name="level">The level.</param>
         /// <param name="message">The message.</param>
         /// <param name="args">The arguments to format <paramref name="message" />.
         /// Can be null (formatting will not occur).</param>
@@ -63,10 +63,10 @@ namespace Apache.Ignite.Log4Net
         /// <param name="category">The logging category name.</param>
         /// <param name="nativeErrorInfo">The native error information.</param>
         /// <param name="ex">The exception. Can be null.</param>
-        public void Log(LogLevel logLevel, string message, object[] args, IFormatProvider formatProvider, 
+        public void Log(LogLevel level, string message, object[] args, IFormatProvider formatProvider, 
             string category, string nativeErrorInfo, Exception ex)
         {
-            var level = ConvertLogLevel(logLevel);
+            var logLevel = ConvertLogLevel(level);
 
             var repo = _log.Logger.Repository;
 
@@ -74,7 +74,7 @@ namespace Apache.Ignite.Log4Net
                 ? (object) message 
                 : new SystemStringFormat(formatProvider, message, args);
 
-            var evt = new LoggingEvent(GetType(), repo, category, level, messageObject, ex);
+            var evt = new LoggingEvent(GetType(), repo, category, logLevel, messageObject, ex);
 
             if (nativeErrorInfo != null)
                 evt.Properties["nativeErrorInfo"] = nativeErrorInfo;
@@ -85,15 +85,15 @@ namespace Apache.Ignite.Log4Net
         /// <summary>
         /// Determines whether the specified log level is enabled.
         /// </summary>
-        /// <param name="logLevel">The level.</param>
+        /// <param name="level">The level.</param>
         /// <returns>
         /// Value indicating whether the specified log level is enabled
         /// </returns>
-        public bool IsEnabled(LogLevel logLevel)
+        public bool IsEnabled(LogLevel level)
         {
-            var level = ConvertLogLevel(logLevel);
+            var logLevel = ConvertLogLevel(level);
 
-            return _log.Logger.IsEnabledFor(level);
+            return _log.Logger.IsEnabledFor(logLevel);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
index 9fc6ffc..520763b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.NLog/Apache.Ignite.NLog.csproj
@@ -20,6 +20,7 @@
     <DefineConstants>DEBUG;TRACE</DefineConstants>
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition=" '$(Configuration)|$(Platform)' == 'Release|AnyCPU' ">
     <DebugType>none</DebugType>
@@ -28,6 +29,7 @@
     <ErrorReport>prompt</ErrorReport>
     <WarningLevel>4</WarningLevel>
     <DocumentationFile>bin\Release\Apache.Ignite.NLog.XML</DocumentationFile>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -35,6 +37,9 @@
   <PropertyGroup>
     <AssemblyOriginatorKeyFile>Apache.Ignite.NLog.snk</AssemblyOriginatorKeyFile>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="NLog, Version=4.0.0.0, Culture=neutral, PublicKeyToken=5120e14c03d0593c, processorArchitecture=MSIL">
       <HintPath>..\packages\NLog.4.3.7\lib\net40\NLog.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index bcaa753..fe83ccd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -2,6 +2,9 @@
 Microsoft Visual Studio Solution File, Format Version 11.00
 # Visual Studio 2010
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core", "Apache.Ignite.Core\Apache.Ignite.Core.csproj", "{4CD2F726-7E2B-46C4-A5BA-057BB82EECB6}"
+	ProjectSection(ProjectDependencies) = postProject
+		{4F7E4917-4612-4B96-9838-025711ADE391} = {4F7E4917-4612-4B96-9838-025711ADE391}
+	EndProjectSection
 EndProject
 Project("{FAE04EC0-301F-11D3-BF4B-00C04F79EFBC}") = "Apache.Ignite.Core.Tests", "Apache.Ignite.Core.Tests\Apache.Ignite.Core.Tests.csproj", "{6A62F66C-DA5B-4FBB-8CE7-A95F740FDC7A}"
 EndProject

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
index e98ddd1..13379f5 100644
--- a/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite/Apache.Ignite.csproj
@@ -23,12 +23,17 @@
     <OutputPath>bin\Debug\</OutputPath>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
   </PropertyGroup>
   <PropertyGroup Condition="'$(Configuration)|$(Platform)' == 'Release|AnyCPU'">
     <OutputPath>bin\Release\</OutputPath>
     <Optimize>true</Optimize>
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
+    <TreatWarningsAsErrors>true</TreatWarningsAsErrors>
+  </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
   </PropertyGroup>
   <ItemGroup>
     <Reference Include="System" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
index 7d0b438..3f7e1dc 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.Examples/Apache.Ignite.Examples.csproj
@@ -35,6 +35,9 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Apache.Ignite.Core">
       <HintPath Condition="Exists('..\..\Apache.Ignite')">..\..\Apache.Ignite\bin\$(Configuration)\Apache.Ignite.Core.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
index 8515bf6..6ea6227 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Apache.Ignite.ExamplesDll.csproj
@@ -11,6 +11,7 @@
     <AssemblyName>Apache.Ignite.ExamplesDll</AssemblyName>
     <TargetFrameworkVersion>v4.0</TargetFrameworkVersion>
     <FileAlignment>512</FileAlignment>
+    <NoWarn>649</NoWarn>
   </PropertyGroup>
   <PropertyGroup>
     <SignAssembly>true</SignAssembly>
@@ -32,6 +33,9 @@
     <PlatformTarget>AnyCPU</PlatformTarget>
     <CodeAnalysisRuleSet>MinimumRecommendedRules.ruleset</CodeAnalysisRuleSet>
   </PropertyGroup>
+  <PropertyGroup>
+    <ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>None</ResolveAssemblyWarnOrErrorOnTargetArchitectureMismatch>
+  </PropertyGroup>
   <ItemGroup>
     <Reference Include="Apache.Ignite.Core">
       <HintPath Condition="Exists('..\..\Apache.Ignite')">..\..\Apache.Ignite\bin\$(Configuration)\Apache.Ignite.Core.dll</HintPath>

http://git-wip-us.apache.org/repos/asf/ignite/blob/87fbb275/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
index 958d91d..7071cd4 100644
--- a/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
+++ b/modules/platforms/dotnet/examples/Apache.Ignite.ExamplesDll/Services/MapService.cs
@@ -30,9 +30,7 @@ namespace Apache.Ignite.ExamplesDll.Services
     public class MapService<TK, TV> : IService
     {
         /** Injected Ignite instance. */
-#pragma warning disable 649
         [InstanceResource] private readonly IIgnite _ignite;
-#pragma warning restore 649
 
         /** Cache. */
         private ICache<TK, TV> _cache;


[29/50] [abbrv] ignite git commit: IGNITE-4424 REPLICATED cache isn't synced across nodes

Posted by sb...@apache.org.
IGNITE-4424 REPLICATED cache isn't synced across nodes


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 82dd912889b0dfca213edb1374c1fa0ed79411fd
Parents: 27ba69b
Author: Anton Vinogradov <av...@apache.org>
Authored: Fri Dec 30 13:41:34 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Jan 16 12:18:14 2017 +0300

----------------------------------------------------------------------
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 ++-
 .../GridNearAtomicSingleUpdateFuture.java       |  44 ++--
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 ++---
 .../AtomicPutAllChangingTopologyTest.java       | 212 +++++++++++++++++++
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 5 files changed, 284 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
index 2fbabaa..c92e0f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
@@ -212,14 +212,18 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
             // Cannot remap.
             remapCnt = 1;
 
-            map(topVer);
+            GridCacheVersion futVer = addAtomicFuture(topVer);
+
+            if (futVer != null)
+                map(topVer, futVer);
         }
     }
 
     /**
      * @param topVer Topology version.
+     * @param futVer Future version
      */
-    protected abstract void map(AffinityTopologyVersion topVer);
+    protected abstract void map(AffinityTopologyVersion topVer, GridCacheVersion futVer);
 
     /**
      * Maps future on ready topology.
@@ -302,7 +306,7 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
      * @param req Request.
      * @param e Error.
      */
-    protected void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
+    protected final void onSendError(GridNearAtomicAbstractUpdateRequest req, IgniteCheckedException e) {
         synchronized (mux) {
             GridNearAtomicUpdateResponse res = new GridNearAtomicUpdateResponse(cctx.cacheId(),
                 req.nodeId(),
@@ -314,4 +318,28 @@ public abstract class GridNearAtomicAbstractUpdateFuture extends GridFutureAdapt
             onResult(req.nodeId(), res, true);
         }
     }
+
+    /**
+     * Adds future prevents topology change before operation complete.
+     * Should be invoked before topology lock released.
+     *
+     * @param topVer Topology version.
+     * @return Future version in case future added.
+     */
+    protected final GridCacheVersion addAtomicFuture(AffinityTopologyVersion topVer) {
+        GridCacheVersion futVer = cctx.versions().next(topVer);
+
+        synchronized (mux) {
+            assert this.futVer == null : this;
+            assert this.topVer == AffinityTopologyVersion.ZERO : this;
+
+            this.topVer = topVer;
+            this.futVer = futVer;
+        }
+
+        if (storeFuture() && !cctx.mvcc().addAtomicFuture(futVer, this))
+            return null;
+
+        return futVer;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index bd231cf..7376aff 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -348,14 +348,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                 @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
                     cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                         @Override public void run() {
-                            try {
-                                AffinityTopologyVersion topVer = fut.get();
-
-                                map(topVer);
-                            }
-                            catch (IgniteCheckedException e) {
-                                onDone(e);
-                            }
+                            mapOnTopology();
                         }
                     });
                 }
@@ -388,7 +381,9 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     @Override protected void mapOnTopology() {
         cache.topology().readLock();
 
-        AffinityTopologyVersion topVer = null;
+        AffinityTopologyVersion topVer;
+
+        GridCacheVersion futVer;
 
         try {
             if (cache.topology().stopping()) {
@@ -410,6 +405,8 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
                 }
 
                 topVer = fut.topologyVersion();
+
+                futVer = addAtomicFuture(topVer);
             }
             else {
                 if (waitTopFut) {
@@ -435,11 +432,12 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             cache.topology().readUnlock();
         }
 
-        map(topVer);
+        if (futVer != null)
+            map(topVer, futVer);
     }
 
     /** {@inheritDoc} */
-    protected void map(AffinityTopologyVersion topVer) {
+    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
         if (F.isEmpty(topNodes)) {
@@ -449,11 +447,6 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             return;
         }
 
-        Exception err = null;
-        GridNearAtomicAbstractUpdateRequest singleReq0 = null;
-
-        GridCacheVersion futVer = cctx.versions().next(topVer);
-
         GridCacheVersion updVer;
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
@@ -470,16 +463,17 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
         else
             updVer = null;
 
+        Exception err = null;
+        GridNearAtomicAbstractUpdateRequest singleReq0 = null;
+
         try {
             singleReq0 = mapSingleUpdate(topVer, futVer, updVer);
 
             synchronized (mux) {
-                assert this.futVer == null : this;
-                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+                assert this.futVer == futVer || (this.isDone() && this.error() != null);
+                assert this.topVer == topVer;
 
-                this.topVer = topVer;
                 this.updVer = updVer;
-                this.futVer = futVer;
 
                 resCnt = 0;
 
@@ -496,14 +490,6 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             return;
         }
 
-        if (storeFuture()) {
-            if (!cctx.mvcc().addAtomicFuture(futVer, this)) {
-                assert isDone() : this;
-
-                return;
-            }
-        }
-
         // Optimize mapping for single key.
         mapSingle(singleReq0.nodeId(), singleReq0);
     }
@@ -511,7 +497,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
     /**
      * @return Future version.
      */
-    GridCacheVersion onFutureDone() {
+    private GridCacheVersion onFutureDone() {
         GridCacheVersion ver0;
 
         GridFutureAdapter<Void> fut0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index cd64117..950e5bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -456,14 +456,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
                 @Override public void apply(final IgniteInternalFuture<AffinityTopologyVersion> fut) {
                     cctx.kernalContext().closure().runLocalSafe(new Runnable() {
                         @Override public void run() {
-                            try {
-                                AffinityTopologyVersion topVer = fut.get();
-
-                                map(topVer, remapKeys);
-                            }
-                            catch (IgniteCheckedException e) {
-                                onDone(e);
-                            }
+                            mapOnTopology();
                         }
                     });
                 }
@@ -497,7 +490,9 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     @Override protected void mapOnTopology() {
         cache.topology().readLock();
 
-        AffinityTopologyVersion topVer = null;
+        AffinityTopologyVersion topVer;
+
+        GridCacheVersion futVer;
 
         try {
             if (cache.topology().stopping()) {
@@ -519,6 +514,8 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
                 }
 
                 topVer = fut.topologyVersion();
+
+                futVer = addAtomicFuture(topVer);
             }
             else {
                 if (waitTopFut) {
@@ -544,7 +541,8 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             cache.topology().readUnlock();
         }
 
-        map(topVer, null);
+        if (futVer != null)
+            map(topVer, futVer, remapKeys);
     }
 
     /**
@@ -602,15 +600,18 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     }
 
     /** {@inheritDoc} */
-    protected void map(AffinityTopologyVersion topVer) {
-        map(topVer, null);
+    @Override protected void map(AffinityTopologyVersion topVer, GridCacheVersion futVer) {
+        map(topVer, futVer, null);
     }
 
     /**
      * @param topVer Topology version.
+     * @param futVer Future ID.
      * @param remapKeys Keys to remap.
      */
-    void map(AffinityTopologyVersion topVer, @Nullable Collection<KeyCacheObject> remapKeys) {
+    void map(AffinityTopologyVersion topVer,
+        GridCacheVersion futVer,
+        @Nullable Collection<KeyCacheObject> remapKeys) {
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
         if (F.isEmpty(topNodes)) {
@@ -620,14 +621,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             return;
         }
 
-        Exception err = null;
-        GridNearAtomicFullUpdateRequest singleReq0 = null;
-        Map<UUID, GridNearAtomicFullUpdateRequest> mappings0 = null;
-
-        int size = keys.size();
-
-        GridCacheVersion futVer = cctx.versions().next(topVer);
-
         GridCacheVersion updVer;
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
@@ -644,6 +637,12 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
         else
             updVer = null;
 
+        Exception err = null;
+        GridNearAtomicFullUpdateRequest singleReq0 = null;
+        Map<UUID, GridNearAtomicFullUpdateRequest> mappings0 = null;
+
+        int size = keys.size();
+
         try {
             if (size == 1 && !fastMap) {
                 assert remapKeys == null || remapKeys.size() == 1;
@@ -676,12 +675,10 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             }
 
             synchronized (mux) {
-                assert this.futVer == null : this;
-                assert this.topVer == AffinityTopologyVersion.ZERO : this;
+                assert this.futVer == futVer || (this.isDone() && this.error() != null);
+                assert this.topVer == topVer;
 
-                this.topVer = topVer;
                 this.updVer = updVer;
-                this.futVer = futVer;
 
                 resCnt = 0;
 
@@ -701,14 +698,6 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
             return;
         }
 
-        if (storeFuture()) {
-            if (!cctx.mvcc().addAtomicFuture(futVer, this)) {
-                assert isDone() : this;
-
-                return;
-            }
-        }
-
         // Optimize mapping for single key.
         if (singleReq0 != null)
             mapSingle(singleReq0.nodeId(), singleReq0);
@@ -725,7 +714,7 @@ public class GridNearAtomicUpdateFuture extends GridNearAtomicAbstractUpdateFutu
     /**
      * @return Future version.
      */
-    GridCacheVersion onFutureDone() {
+    private GridCacheVersion onFutureDone() {
         GridCacheVersion ver0;
 
         GridFutureAdapter<Void> fut0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
new file mode 100644
index 0000000..878cb17
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/AtomicPutAllChangingTopologyTest.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cache.affinity.fair.FairAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CachePeekMode.BACKUP;
+import static org.apache.ignite.cache.CachePeekMode.PRIMARY;
+import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/** */
+public class AtomicPutAllChangingTopologyTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES_CNT = 3;
+
+    /** */
+    public static final String CACHE_NAME = "test-cache";
+
+    /** */
+    private static final int CACHE_SIZE = 20_000;
+
+    /** */
+    private static volatile CountDownLatch FILLED_LATCH;
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfig() {
+        return new CacheConfiguration<Integer, Integer>()
+            .setAtomicityMode(ATOMIC)
+            .setCacheMode(REPLICATED)
+            .setAffinity(new FairAffinityFunction(false, 1))
+            .setWriteSynchronizationMode(FULL_SYNC)
+            .setRebalanceMode(SYNC)
+            .setName(CACHE_NAME);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutAllOnChangingTopology() throws Exception {
+        List<IgniteInternalFuture> futs = new LinkedList<>();
+
+        for (int i = 1; i < NODES_CNT; i++)
+            futs.add(startNodeAsync(i));
+
+        futs.add(startSeedNodeAsync());
+
+        boolean failed = false;
+
+        for (IgniteInternalFuture fut : futs) {
+            try {
+                fut.get();
+            }
+            catch (Throwable th) {
+                log.error("Check failed.", th);
+
+                failed = true;
+            }
+        }
+
+        if (failed)
+            throw new RuntimeException("Test Failed.");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        FILLED_LATCH = new CountDownLatch(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @return Future.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgniteInternalFuture startSeedNodeAsync() throws IgniteCheckedException {
+        return GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Boolean call() throws Exception {
+                Ignite node = startGrid(0);
+
+                log.info("Creating cache.");
+
+                IgniteCache<Integer, Integer> cache = node.getOrCreateCache(cacheConfig());
+
+                log.info("Created cache.");
+
+                Map<Integer, Integer> data = new HashMap<>(CACHE_SIZE);
+
+                for (int i = 0; i < CACHE_SIZE; i++)
+                    data.put(i, i);
+
+                log.info("Filling.");
+
+                cache.putAll(data);
+
+                log.info("Filled.");
+
+                FILLED_LATCH.countDown();
+
+                checkCacheState(node, cache);
+
+                return true;
+            }
+        });
+    }
+
+    /**
+     * @param nodeId Node index.
+     * @return Future.
+     * @throws IgniteCheckedException If failed.
+     */
+    private IgniteInternalFuture startNodeAsync(final int nodeId) throws IgniteCheckedException {
+        return GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Boolean call() throws Exception {
+                Ignite node = startGrid(nodeId);
+
+                log.info("Getting cache.");
+
+                IgniteCache<Integer, Integer> cache = node.getOrCreateCache(cacheConfig());
+
+                log.info("Got cache.");
+
+                FILLED_LATCH.await();
+
+                log.info("Got Filled.");
+
+                cache.put(1, nodeId);
+
+                checkCacheState(node, cache);
+
+                return true;
+            }
+        });
+    }
+
+    /**
+     * @param node Node.
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void checkCacheState(Ignite node, IgniteCache<Integer, Integer> cache) throws Exception {
+        int locSize = cache.localSize(PRIMARY, BACKUP);
+        int locSize2 = -1;
+
+        if (locSize != CACHE_SIZE) {
+            U.sleep(5000);
+
+            // Rechecking.
+            locSize2 = cache.localSize(PRIMARY, BACKUP);
+        }
+
+        assertEquals("Wrong cache size on node [node=" + node.configuration().getGridName() +
+            ", expected= " + CACHE_SIZE +
+            ", actual=" + locSize +
+            ", actual2=" + locSize2 + "]",
+            locSize, CACHE_SIZE);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/82dd9128/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index 26cea39..986b8d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtR
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheTxNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridNearCacheTxNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteAtomicLongChangingTopologySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.AtomicPutAllChangingTopologyTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientInvalidPartitionHandlingSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicClientRemoveFailureTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridCacheAtomicInvalidPartitionHandlingSelfTest;
@@ -97,6 +98,8 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheTxNodeFailureSelfTest.class);
         suite.addTestSuite(GridNearCacheTxNodeFailureSelfTest.class);
 
+        suite.addTestSuite(AtomicPutAllChangingTopologyTest.class);
+
         return suite;
     }
 }


[36/50] [abbrv] ignite git commit: IGNITE-4045 .NET: Support DML API

Posted by sb...@apache.org.
IGNITE-4045 .NET: Support DML API

This closes #1309


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

Branch: refs/heads/ignite-gg-11810-1
Commit: b7908d7a65f07615f2ff183e107c5002658bd511
Parents: d6d42c2
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jan 16 17:48:08 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 16 17:48:08 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       | 128 +++++++-
 .../Apache.Ignite.Core.Tests.csproj             |   3 +
 .../Binary/BinaryBuilderSelfTest.cs             | 159 ++++++----
 .../BinaryBuilderSelfTestArrayIdentity.cs       |  34 +++
 .../Binary/BinaryEqualityComparerTest.cs        | 279 +++++++++++++++++
 .../Binary/IO/BinaryStreamsTest.cs              |  19 ++
 .../Cache/CacheConfigurationTest.cs             |   5 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          | 296 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        |  46 ++-
 .../IgniteConfigurationTest.cs                  |  28 ++
 .../Apache.Ignite.Core.csproj                   |   4 +
 .../Binary/BinaryArrayEqualityComparer.cs       | 149 ++++++++++
 .../Binary/BinaryConfiguration.cs               |  24 ++
 .../Binary/BinaryTypeConfiguration.cs           |  14 +
 .../Cache/Configuration/QueryEntity.cs          |  33 ++-
 .../Cache/Configuration/QueryField.cs           |   6 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  85 ++++--
 .../IgniteConfigurationSection.xsd              |  19 ++
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |  28 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |  99 +++++++
 .../Impl/Binary/BinaryFieldEqualityComparer.cs  | 138 +++++++++
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  21 +-
 .../Impl/Binary/BinaryObject.cs                 |  31 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  62 +++-
 .../Impl/Binary/BinaryObjectHeader.cs           |  21 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  22 ++
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   6 +
 .../Impl/Binary/BinarySystemHandlers.cs         |   6 +-
 .../Impl/Binary/BinaryWriter.cs                 |  11 +-
 .../Impl/Binary/DateTimeHolder.cs               |  15 +-
 .../Impl/Binary/IBinaryEqualityComparer.cs      |  53 ++++
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   5 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |   9 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  13 +
 .../Impl/Binary/Io/IBinaryStream.cs             |  25 ++
 .../Impl/Binary/Marshaller.cs                   |  22 +-
 .../Impl/Binary/SerializableObjectHolder.cs     |  16 +
 .../Common/IgniteConfigurationXmlSerializer.cs  |   5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |  16 +
 39 files changed, 1803 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index f845675..c0fde97 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -17,8 +17,12 @@
 
 package org.apache.ignite.internal.processors.platform.utils;
 
+import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+import org.apache.ignite.binary.BinaryIdentityResolver;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMemoryMode;
@@ -68,6 +72,8 @@ import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.HashSet;
 
 /**
  * Configuration utils.
@@ -428,14 +434,25 @@ public class PlatformConfigurationUtils {
 
         // Fields
         int cnt = in.readInt();
+        Set<String> keyFields = new HashSet<>(cnt);
 
         if (cnt > 0) {
             LinkedHashMap<String, String> fields = new LinkedHashMap<>(cnt);
 
-            for (int i = 0; i < cnt; i++)
-                fields.put(in.readString(), in.readString());
+            for (int i = 0; i < cnt; i++) {
+                String fieldName = in.readString();
+                String fieldType = in.readString();
+
+                fields.put(fieldName, fieldType);
+
+                if (in.readBoolean())
+                    keyFields.add(fieldName);
+            }
 
             res.setFields(fields);
+
+            if (!keyFields.isEmpty())
+                res.setKeyFields(keyFields);
         }
 
         // Aliases
@@ -539,11 +556,29 @@ public class PlatformConfigurationUtils {
             cfg.setCommunicationSpi(comm);
         }
 
-        if (in.readBoolean()) {
+        if (in.readBoolean()) {  // binary config is present
             if (cfg.getBinaryConfiguration() == null)
                 cfg.setBinaryConfiguration(new BinaryConfiguration());
 
-            cfg.getBinaryConfiguration().setCompactFooter(in.readBoolean());
+            if (in.readBoolean())  // compact footer is set
+                cfg.getBinaryConfiguration().setCompactFooter(in.readBoolean());
+
+            int typeCnt = in.readInt();
+
+            if (typeCnt > 0) {
+                Collection<BinaryTypeConfiguration> types = new ArrayList<>(typeCnt);
+
+                for (int i = 0; i < typeCnt; i++) {
+                    BinaryTypeConfiguration type = new BinaryTypeConfiguration(in.readString());
+
+                    type.setEnum(in.readBoolean());
+                    type.setIdentityResolver(readBinaryIdentityResolver(in));
+
+                    types.add(type);
+                }
+
+                cfg.getBinaryConfiguration().setTypeConfigurations(types);
+            }
         }
 
         int attrCnt = in.readInt();
@@ -812,11 +847,14 @@ public class PlatformConfigurationUtils {
         LinkedHashMap<String, String> fields = queryEntity.getFields();
 
         if (fields != null) {
+            Set<String> keyFields = queryEntity.getKeyFields();
+
             writer.writeInt(fields.size());
 
             for (Map.Entry<String, String> field : fields.entrySet()) {
                 writer.writeString(field.getKey());
                 writer.writeString(field.getValue());
+                writer.writeBoolean(keyFields != null && keyFields.contains(field.getKey()));
             }
         }
         else
@@ -941,11 +979,29 @@ public class PlatformConfigurationUtils {
             w.writeBoolean(false);
 
         BinaryConfiguration bc = cfg.getBinaryConfiguration();
-        w.writeBoolean(bc != null);
 
-        if (bc != null)
+        if (bc != null) {
+            w.writeBoolean(true);  // binary config exists
+            w.writeBoolean(true);  // compact footer is set
             w.writeBoolean(bc.isCompactFooter());
 
+            Collection<BinaryTypeConfiguration> types = bc.getTypeConfigurations();
+
+            if (types != null) {
+                w.writeInt(types.size());
+
+                for (BinaryTypeConfiguration type : types) {
+                    w.writeString(type.getTypeName());
+                    w.writeBoolean(type.isEnum());
+                    writeBinaryIdentityResolver(w, type.getIdentityResolver());
+                }
+            }
+            else
+                w.writeInt(0);
+        }
+        else
+            w.writeBoolean(false);
+
         Map<String, ?> attrs = cfg.getUserAttributes();
 
         if (attrs != null) {
@@ -1117,6 +1173,66 @@ public class PlatformConfigurationUtils {
     }
 
     /**
+     * Reads resolver
+     *
+     * @param r Reader.
+     * @return Resolver.
+     */
+    private static BinaryIdentityResolver readBinaryIdentityResolver(BinaryRawReader r) {
+        int type = r.readByte();
+
+        switch (type) {
+            case 0:
+                return null;
+
+            case 1:
+                return new BinaryArrayIdentityResolver();
+
+            case 2:
+                int cnt = r.readInt();
+
+                String[] fields = new String[cnt];
+
+                for (int i = 0; i < cnt; i++)
+                    fields[i] = r.readString();
+
+                return new BinaryFieldIdentityResolver().setFieldNames(fields);
+
+            default:
+                assert false;
+                return null;
+        }
+    }
+
+    /**
+     * Writes the resolver.
+     *
+     * @param w Writer.
+     * @param resolver Resolver.
+     */
+    private static void writeBinaryIdentityResolver(BinaryRawWriter w, BinaryIdentityResolver resolver) {
+        if (resolver instanceof BinaryArrayIdentityResolver)
+            w.writeByte((byte)1);
+        else if (resolver instanceof BinaryFieldIdentityResolver) {
+            w.writeByte((byte)2);
+
+            String[] fields = ((BinaryFieldIdentityResolver)resolver).getFieldNames();
+
+            if (fields != null) {
+                w.writeInt(fields.length);
+
+                for (String field : fields)
+                    w.writeString(field);
+            }
+            else
+                w.writeInt(0);
+        }
+        else {
+            w.writeByte((byte)0);
+        }
+    }
+
+    /**
      * Private constructor.
      */
     private PlatformConfigurationUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 78a08d2..e09c682 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -67,6 +67,8 @@
     <Reference Include="System.Xml.Linq" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Binary\BinaryBuilderSelfTestArrayIdentity.cs" />
+    <Compile Include="Binary\BinaryEqualityComparerTest.cs" />
     <Compile Include="Binary\BinaryReaderWriterTest.cs" />
     <Compile Include="Binary\IO\BinaryStreamsTest.cs" />
     <Compile Include="Binary\JavaTypeMappingTest.cs" />
@@ -76,6 +78,7 @@
     <Compile Include="Cache\CacheMetricsTest.cs" />
     <Compile Include="Cache\CacheResultTest.cs" />
     <Compile Include="Cache\CacheSwapSpaceTest.cs" />
+    <Compile Include="Cache\Query\CacheDmlQueriesTest.cs" />
     <Compile Include="Cache\CacheAbstractTransactionalTest.cs" />
     <Compile Include="Cache\Store\CacheStoreAdapterTest.cs" />
     <Compile Include="Collections\MultiValueDictionaryTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
index c280255..d6551b5 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTest.cs
@@ -58,33 +58,57 @@ namespace Apache.Ignite.Core.Tests.Binary
                 {
                     TypeConfigurations = new List<BinaryTypeConfiguration>
                     {
-                        new BinaryTypeConfiguration(typeof (Empty)),
-                        new BinaryTypeConfiguration(typeof (Primitives)),
-                        new BinaryTypeConfiguration(typeof (PrimitiveArrays)),
-                        new BinaryTypeConfiguration(typeof (StringDateGuidEnum)),
-                        new BinaryTypeConfiguration(typeof (WithRaw)),
-                        new BinaryTypeConfiguration(typeof (MetaOverwrite)),
-                        new BinaryTypeConfiguration(typeof (NestedOuter)),
-                        new BinaryTypeConfiguration(typeof (NestedInner)),
-                        new BinaryTypeConfiguration(typeof (MigrationOuter)),
-                        new BinaryTypeConfiguration(typeof (MigrationInner)),
-                        new BinaryTypeConfiguration(typeof (InversionOuter)),
-                        new BinaryTypeConfiguration(typeof (InversionInner)),
-                        new BinaryTypeConfiguration(typeof (CompositeOuter)),
-                        new BinaryTypeConfiguration(typeof (CompositeInner)),
-                        new BinaryTypeConfiguration(typeof (CompositeArray)),
-                        new BinaryTypeConfiguration(typeof (CompositeContainer)),
-                        new BinaryTypeConfiguration(typeof (ToBinary)),
-                        new BinaryTypeConfiguration(typeof (Remove)),
-                        new BinaryTypeConfiguration(typeof (RemoveInner)),
-                        new BinaryTypeConfiguration(typeof (BuilderInBuilderOuter)),
-                        new BinaryTypeConfiguration(typeof (BuilderInBuilderInner)),
-                        new BinaryTypeConfiguration(typeof (BuilderCollection)),
-                        new BinaryTypeConfiguration(typeof (BuilderCollectionItem)),
-                        new BinaryTypeConfiguration(typeof (DecimalHolder)),
-                        new BinaryTypeConfiguration(TypeEmpty),
-                        new BinaryTypeConfiguration(typeof(TestEnumRegistered)),
+                        new BinaryTypeConfiguration(typeof(Empty)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(Primitives)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(PrimitiveArrays)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(StringDateGuidEnum))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(WithRaw)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(MetaOverwrite)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(NestedOuter)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(NestedInner)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(MigrationOuter)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(MigrationInner)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(InversionOuter)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(InversionInner)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(CompositeOuter)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(CompositeInner)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(CompositeArray)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(CompositeContainer))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(ToBinary)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(Remove)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(RemoveInner)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(BuilderInBuilderOuter))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(BuilderInBuilderInner))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(BuilderCollection))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(BuilderCollectionItem))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
+                        new BinaryTypeConfiguration(typeof(DecimalHolder)) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(TypeEmpty) {EqualityComparer = GetIdentityResolver()},
+                        new BinaryTypeConfiguration(typeof(TestEnumRegistered))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        },
                         new BinaryTypeConfiguration(typeof(NameMapperTestType))
+                        {
+                            EqualityComparer = GetIdentityResolver()
+                        }
                     },
                     DefaultIdMapper = new IdMapper(),
                     DefaultNameMapper = new NameMapper(),
@@ -106,6 +130,14 @@ namespace Apache.Ignite.Core.Tests.Binary
         }
 
         /// <summary>
+        /// Gets the identity resolver.
+        /// </summary>
+        protected virtual IEqualityComparer<IBinaryObject> GetIdentityResolver()
+        {
+            return null;
+        }
+
+        /// <summary>
         /// Tear down routine.
         /// </summary>
         [TestFixtureTearDown]
@@ -535,7 +567,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             IBinaryObject binObj = _grid.GetBinary().GetBuilder(typeof(Empty)).Build();
 
             Assert.IsNotNull(binObj);
-            Assert.AreEqual(0, binObj.GetHashCode());
+            Assert.AreEqual(GetIdentityResolver() == null ? 0 : 1, binObj.GetHashCode());
 
             IBinaryType meta = binObj.GetBinaryType();
 
@@ -557,7 +589,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             IBinaryObject binObj = _grid.GetBinary().GetBuilder(TypeEmpty).Build();
 
             Assert.IsNotNull(binObj);
-            Assert.AreEqual(0, binObj.GetHashCode());
+            Assert.AreEqual(GetIdentityResolver() == null ? 0 : 1, binObj.GetHashCode());
 
             IBinaryType meta = binObj.GetBinaryType();
 
@@ -602,7 +634,9 @@ namespace Apache.Ignite.Core.Tests.Binary
             var obj2 = bin.GetBuilder("myType").SetStringField("str", "foo").SetIntField("int", 1).Build();
 
             Assert.AreEqual(obj1, obj2);
-            Assert.AreEqual(obj1.GetHashCode(), obj2.GetHashCode());
+
+            Assert.AreEqual(0, obj1.GetHashCode());
+            Assert.AreEqual(0, obj2.GetHashCode());
 
             Assert.IsTrue(Regex.IsMatch(obj1.ToString(), @"myType \[idHash=[0-9]+, str=foo, int=1\]"));
         }
@@ -630,7 +664,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             CheckPrimitiveFields1(binObj);
 
             // Specific setter methods.
-            binObj = _grid.GetBinary().GetBuilder(typeof(Primitives))
+            var binObj2 = _grid.GetBinary().GetBuilder(typeof(Primitives))
                 .SetByteField("fByte", 1)
                 .SetBooleanField("fBool", true)
                 .SetShortField("fShort", 2)
@@ -643,7 +677,11 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetHashCode(100)
                 .Build();
 
-            CheckPrimitiveFields1(binObj);
+            CheckPrimitiveFields1(binObj2);
+
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
 
             // Overwrite with generic methods.
             binObj = binObj.ToBuilder()
@@ -656,13 +694,12 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetField<float>("fFloat", 11)
                 .SetField<double>("fDouble", 12)
                 .SetField("fDecimal", 13.13m)
-                .SetHashCode(200)
                 .Build();
 
             CheckPrimitiveFields2(binObj);
 
             // Overwrite with specific methods.
-            binObj = binObj.ToBuilder()
+            binObj2 = binObj.ToBuilder()
                 .SetByteField("fByte", 7)
                 .SetBooleanField("fBool", false)
                 .SetShortField("fShort", 8)
@@ -672,10 +709,13 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetFloatField("fFloat", 11)
                 .SetDoubleField("fDouble", 12)
                 .SetDecimalField("fDecimal", 13.13m)
-                .SetHashCode(200)
                 .Build();
 
             CheckPrimitiveFields2(binObj);
+
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
         }
 
         /// <summary>
@@ -729,8 +769,6 @@ namespace Apache.Ignite.Core.Tests.Binary
         /// </summary>
         private static void CheckPrimitiveFields2(IBinaryObject binObj)
         {
-            Assert.AreEqual(200, binObj.GetHashCode());
-
             Assert.AreEqual(7, binObj.GetField<byte>("fByte"));
             Assert.AreEqual(false, binObj.GetField<bool>("fBool"));
             Assert.AreEqual(8, binObj.GetField<short>("fShort"));
@@ -777,7 +815,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             CheckPrimitiveArrayFields1(binObj);
 
             // Specific setters.
-            binObj = _grid.GetBinary().GetBuilder(typeof(PrimitiveArrays))
+            var binObj2 = _grid.GetBinary().GetBuilder(typeof(PrimitiveArrays))
                 .SetByteArrayField("fByte", new byte[] {1})
                 .SetBooleanArrayField("fBool", new[] {true})
                 .SetShortArrayField("fShort", new short[] {2})
@@ -789,7 +827,13 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetDecimalArrayField("fDecimal", new decimal?[] {7.7m})
                 .SetHashCode(100)
                 .Build();
-            
+
+            CheckPrimitiveArrayFields1(binObj2);
+
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
+
             // Overwrite with generic setter.
             binObj = _grid.GetBinary().GetBuilder(binObj)
                 .SetField("fByte", new byte[] { 7 })
@@ -801,13 +845,12 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetField("fFloat", new float[] { 11 })
                 .SetField("fDouble", new double[] { 12 })
                 .SetField("fDecimal", new decimal?[] { 13.13m })
-                .SetHashCode(200)
                 .Build();
 
             CheckPrimitiveArrayFields2(binObj);
 
             // Overwrite with specific setters.
-            binObj = _grid.GetBinary().GetBuilder(binObj)
+            binObj2 = _grid.GetBinary().GetBuilder(binObj)
                 .SetByteArrayField("fByte", new byte[] { 7 })
                 .SetBooleanArrayField("fBool", new[] { false })
                 .SetShortArrayField("fShort", new short[] { 8 })
@@ -817,10 +860,13 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetFloatArrayField("fFloat", new float[] { 11 })
                 .SetDoubleArrayField("fDouble", new double[] { 12 })
                 .SetDecimalArrayField("fDecimal", new decimal?[] { 13.13m })
-                .SetHashCode(200)
                 .Build();
 
             CheckPrimitiveArrayFields2(binObj);
+            
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
         }
 
         /// <summary>
@@ -874,8 +920,6 @@ namespace Apache.Ignite.Core.Tests.Binary
         /// </summary>
         private static void CheckPrimitiveArrayFields2(IBinaryObject binObj)
         {
-            Assert.AreEqual(200, binObj.GetHashCode());
-
             Assert.AreEqual(new byte[] { 7 }, binObj.GetField<byte[]>("fByte"));
             Assert.AreEqual(new[] { false }, binObj.GetField<bool[]>("fBool"));
             Assert.AreEqual(new short[] { 8 }, binObj.GetField<short[]>("fShort"));
@@ -910,7 +954,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             Guid? nGuid = Guid.NewGuid();
 
             // Generic setters.
-            IBinaryObject binObj = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
+            var binObj = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
                 .SetField("fStr", "str")
                 .SetField("fNDate", nDate)
                 .SetTimestampField("fNTimestamp", nDate)
@@ -927,7 +971,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             CheckStringDateGuidEnum1(binObj, nDate, nGuid);
 
             // Specific setters.
-            binObj = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
+            var binObj2 = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
                 .SetStringField("fStr", "str")
                 .SetField("fNDate", nDate)
                 .SetTimestampField("fNTimestamp", nDate)
@@ -941,7 +985,11 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetHashCode(100)
                 .Build();
 
-            CheckStringDateGuidEnum1(binObj, nDate, nGuid);
+            CheckStringDateGuidEnum1(binObj2, nDate, nGuid);
+
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
 
             // Overwrite.
             nDate = DateTime.Now.ToUniversalTime();
@@ -958,13 +1006,12 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetTimestampArrayField("fTimestampArr", new[] { nDate })
                 .SetField("fGuidArr", new[] { nGuid })
                 .SetField("fEnumArr", new[] { TestEnum.Two })
-                .SetHashCode(200)
                 .Build();
 
             CheckStringDateGuidEnum2(binObj, nDate, nGuid);
 
             // Overwrite with specific setters
-            binObj = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
+            binObj2 = _grid.GetBinary().GetBuilder(typeof(StringDateGuidEnum))
                 .SetStringField("fStr", "str2")
                 .SetField("fNDate", nDate)
                 .SetTimestampField("fNTimestamp", nDate)
@@ -975,10 +1022,13 @@ namespace Apache.Ignite.Core.Tests.Binary
                 .SetTimestampArrayField("fTimestampArr", new[] { nDate })
                 .SetGuidArrayField("fGuidArr", new[] { nGuid })
                 .SetEnumArrayField("fEnumArr", new[] { TestEnum.Two })
-                .SetHashCode(200)
                 .Build();
 
-            CheckStringDateGuidEnum2(binObj, nDate, nGuid);
+            CheckStringDateGuidEnum2(binObj2, nDate, nGuid);
+
+            // Check equality.
+            Assert.AreEqual(binObj, binObj2);
+            Assert.AreEqual(binObj.GetHashCode(), binObj2.GetHashCode());
         }
 
         /// <summary>
@@ -1074,8 +1124,6 @@ namespace Apache.Ignite.Core.Tests.Binary
         /// </summary>
         private static void CheckStringDateGuidEnum2(IBinaryObject binObj, DateTime? nDate, Guid? nGuid)
         {
-            Assert.AreEqual(200, binObj.GetHashCode());
-
             Assert.AreEqual("str2", binObj.GetField<string>("fStr"));
             Assert.AreEqual(nDate, binObj.GetField<DateTime?>("fNDate"));
             Assert.AreEqual(nDate, binObj.GetField<DateTime?>("fNTimestamp"));
@@ -1432,7 +1480,8 @@ namespace Apache.Ignite.Core.Tests.Binary
 
             IBinaryObjectBuilder builder = _grid.GetBinary().GetBuilder(typeof(MigrationOuter));
 
-            builder.SetHashCode(outer.GetHashCode());
+            if (GetIdentityResolver() == null)
+                builder.SetHashCode(outer.GetHashCode());
 
             builder.SetField<object>("inner1", inner);
             builder.SetField<object>("inner2", inner);
@@ -1642,6 +1691,7 @@ namespace Apache.Ignite.Core.Tests.Binary
             }
 
             Assert.AreEqual(binEnums[0], binEnums[1]);
+            Assert.AreEqual(binEnums[0].GetHashCode(), binEnums[1].GetHashCode());
         }
 
         /// <summary>
@@ -1659,6 +1709,9 @@ namespace Apache.Ignite.Core.Tests.Binary
         [Test]
         public void TestRemoteBinaryMode()
         {
+            if (GetIdentityResolver() != null)
+                return;  // When identity resolver is set, it is required to have the same config on all nodes.
+
             var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
             {
                 GridName = "grid2",

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestArrayIdentity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestArrayIdentity.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestArrayIdentity.cs
new file mode 100644
index 0000000..b5e767c
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryBuilderSelfTestArrayIdentity.cs
@@ -0,0 +1,34 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System.Collections.Generic;
+    using Apache.Ignite.Core.Binary;
+
+    /// <summary>
+    /// Tests with array equality comparer (identity resolver).
+    /// </summary>
+    public class BinaryBuilderSelfTestArrayIdentity : BinaryBuilderSelfTest
+    {
+        /** <inheritdoc /> */
+        protected override IEqualityComparer<IBinaryObject> GetIdentityResolver()
+        {
+            return new BinaryArrayEqualityComparer();
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryEqualityComparerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryEqualityComparerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryEqualityComparerTest.cs
new file mode 100644
index 0000000..f0550a8
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryEqualityComparerTest.cs
@@ -0,0 +1,279 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Binary
+{
+    using System.Collections.Generic;
+    using System.Diagnostics.CodeAnalysis;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Equality comparers test.
+    /// </summary>
+    public class BinaryEqualityComparerTest
+    {
+        /// <summary>
+        /// Tests common public methods logic.
+        /// </summary>
+        [Test]
+        [SuppressMessage("ReSharper", "ReturnValueOfPureMethodIsNotUsed")]
+        public void TestPublicMethods()
+        {
+            var cmps = new IEqualityComparer<IBinaryObject>[]
+            {
+                new BinaryArrayEqualityComparer()
+                //new BinaryFieldEqualityComparer()
+            };
+
+            var obj = GetBinaryObject(1, "x", 0);
+
+            foreach (var cmp in cmps)
+            {
+                Assert.IsTrue(cmp.Equals(null, null));
+                Assert.IsTrue(cmp.Equals(obj, obj));
+
+                Assert.IsFalse(cmp.Equals(obj, null));
+                Assert.IsFalse(cmp.Equals(null, obj));
+
+                Assert.AreEqual(0, cmp.GetHashCode(null));
+                Assert.AreNotEqual(0, cmp.GetHashCode(obj));
+            }
+        }
+
+        /// <summary>
+        /// Tests the custom comparer.
+        /// </summary>
+        [Test]
+        public void TestCustomComparer()
+        {
+            var ex = Assert.Throws<IgniteException>(() => Ignition.Start(
+                new IgniteConfiguration(TestUtils.GetTestConfiguration())
+                {
+                    BinaryConfiguration = new BinaryConfiguration
+                    {
+                        TypeConfigurations = new[]
+                        {
+                            new BinaryTypeConfiguration(typeof(Foo))
+                            {
+                                EqualityComparer = new MyComparer()
+                            }
+                        }
+                    }
+                }));
+
+            Assert.AreEqual("Unsupported IEqualityComparer<IBinaryObject> implementation: " +
+                            "Apache.Ignite.Core.Tests.Binary.BinaryEqualityComparerTest+MyComparer. " +
+                            "Only predefined implementations are supported.", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests the array comparer.
+        /// </summary>
+        [Test]
+        public void TestArrayComparer()
+        {
+            var cmp = (IBinaryEqualityComparer) new BinaryArrayEqualityComparer();
+
+            var ms = new BinaryHeapStream(10);
+
+            Assert.AreEqual(1, cmp.GetHashCode(ms, 0, 0, null, 0, null, null));
+
+            ms.WriteByte(1);
+            Assert.AreEqual(31 + 1, cmp.GetHashCode(ms, 0, 1, null, 0, null, null));
+
+            ms.WriteByte(3);
+            Assert.AreEqual((31 + 1) * 31 + 3, cmp.GetHashCode(ms, 0, 2, null, 0, null, null));
+        }
+
+        /// <summary>
+        /// Tests public methods of array comparer.
+        /// </summary>
+        [Test]
+        public void TestArrayComparerPublic()
+        {
+            var cmp = new BinaryArrayEqualityComparer();
+
+            var obj1 = GetBinaryObject(1, "foo", 11);
+            var obj2 = GetBinaryObject(1, "bar", 11);
+            var obj3 = GetBinaryObject(2, "foo", 11);
+            var obj4 = GetBinaryObject(2, "bar", 11);
+            var obj5 = GetBinaryObject(1, "foo", 11);
+            var obj6 = GetBinaryObject(1, "foo", 12);
+
+            // Equals.
+            Assert.IsTrue(cmp.Equals(obj1, obj1));
+            Assert.IsTrue(cmp.Equals(obj1, obj5));
+            Assert.IsFalse(cmp.Equals(obj1, obj2));
+            Assert.IsFalse(cmp.Equals(obj1, obj3));
+            Assert.IsFalse(cmp.Equals(obj1, obj4));
+            Assert.IsFalse(cmp.Equals(obj1, obj6));
+
+            Assert.IsTrue(cmp.Equals(obj2, obj2));
+            Assert.IsFalse(cmp.Equals(obj2, obj5));
+            Assert.IsFalse(cmp.Equals(obj2, obj3));
+            Assert.IsFalse(cmp.Equals(obj2, obj4));
+            Assert.IsFalse(cmp.Equals(obj2, obj6));
+
+            Assert.IsTrue(cmp.Equals(obj3, obj3));
+            Assert.IsFalse(cmp.Equals(obj3, obj5));
+            Assert.IsFalse(cmp.Equals(obj3, obj4));
+            Assert.IsFalse(cmp.Equals(obj3, obj6));
+
+            Assert.IsTrue(cmp.Equals(obj4, obj4));
+            Assert.IsFalse(cmp.Equals(obj4, obj5));
+            Assert.IsFalse(cmp.Equals(obj4, obj6));
+
+            Assert.IsTrue(cmp.Equals(obj5, obj5));
+            Assert.IsFalse(cmp.Equals(obj5, obj6));
+
+            // BinaryObject.GetHashCode.
+            Assert.AreEqual(1934949494, obj1.GetHashCode());
+            Assert.AreEqual(-2013102781, obj2.GetHashCode());
+            Assert.AreEqual(1424415317, obj3.GetHashCode());
+            Assert.AreEqual(1771330338, obj4.GetHashCode());
+            Assert.AreEqual(obj1.GetHashCode(), obj5.GetHashCode());
+            Assert.AreEqual(1934979285, cmp.GetHashCode(obj6));
+
+            // Comparer.GetHashCode.
+            Assert.AreEqual(2001751043, cmp.GetHashCode(GetBinaryObject(0, null, 0)));
+            Assert.AreEqual(194296580, cmp.GetHashCode(GetBinaryObject(1, null, 0)));
+            Assert.AreEqual(1934949494, cmp.GetHashCode(obj1));
+            Assert.AreEqual(-2013102781, cmp.GetHashCode(obj2));
+            Assert.AreEqual(1424415317, cmp.GetHashCode(obj3));
+            Assert.AreEqual(1771330338, cmp.GetHashCode(obj4));
+            Assert.AreEqual(cmp.GetHashCode(obj1), cmp.GetHashCode(obj5));
+            Assert.AreEqual(1934979285, cmp.GetHashCode(obj6));
+
+            // GetHashCode consistency.
+            foreach (var obj in new[] {obj1, obj2, obj3, obj4, obj5, obj6})
+                Assert.AreEqual(obj.GetHashCode(), cmp.GetHashCode(obj));
+        }
+
+        /// <summary>
+        /// Tests the field comparer.
+        /// </summary>
+        [Test]
+        public void TestFieldComparer()
+        {
+            var marsh = new Marshaller(new BinaryConfiguration
+            {
+                TypeConfigurations = new[]
+                {
+                    new BinaryTypeConfiguration(typeof(Foo))
+                    {
+                        EqualityComparer = new BinaryFieldEqualityComparer("Name", "Id")
+                    }
+                }
+            });
+
+            var val = new Foo {Id = 58, Name = "John"};
+            var binObj = marsh.Unmarshal<IBinaryObject>(marsh.Marshal(val), BinaryMode.ForceBinary);
+            var expHash = val.Name.GetHashCode() * 31 + val.Id.GetHashCode();
+            Assert.AreEqual(expHash, binObj.GetHashCode());
+
+            val = new Foo {Id = 95};
+            binObj = marsh.Unmarshal<IBinaryObject>(marsh.Marshal(val), BinaryMode.ForceBinary);
+            expHash = val.Id.GetHashCode();
+            Assert.AreEqual(expHash, binObj.GetHashCode());
+        }
+
+        /// <summary>
+        /// Tests the field comparer validation.
+        /// </summary>
+        [Test]
+        public void TestFieldComparerValidation()
+        {
+            var ex = Assert.Throws<IgniteException>(() => Ignition.Start(
+                new IgniteConfiguration(TestUtils.GetTestConfiguration())
+                {
+                    BinaryConfiguration = new BinaryConfiguration
+                    {
+                        TypeConfigurations = new[]
+                        {
+                            new BinaryTypeConfiguration(typeof(Foo))
+                            {
+                                EqualityComparer = new BinaryFieldEqualityComparer()
+                            }
+                        }
+                    }
+                }));
+
+            Assert.AreEqual("BinaryFieldEqualityComparer.FieldNames can not be null or empty.", ex.Message);
+        }
+
+        /// <summary>
+        /// Gets the binary object.
+        /// </summary>
+        private static IBinaryObject GetBinaryObject(int id, string name, int raw)
+        {
+            var marsh = new Marshaller(new BinaryConfiguration
+            {
+                TypeConfigurations = new[]
+                {
+                    new BinaryTypeConfiguration(typeof(Foo))
+                    {
+                        EqualityComparer = new BinaryArrayEqualityComparer()
+                    }
+                }
+            });
+
+            var bytes = marsh.Marshal(new Foo {Id = id, Name = name, Raw = raw});
+
+            return marsh.Unmarshal<IBinaryObject>(bytes, BinaryMode.ForceBinary);
+        }
+
+        private class Foo : IBinarizable
+        {
+            public int Id { get; set; }
+            public string Name { get; set; }
+            public int Raw { get; set; }
+
+            public void WriteBinary(IBinaryWriter writer)
+            {
+                writer.WriteInt("id", Id);
+                writer.WriteString("name", Name);
+
+                writer.GetRawWriter().WriteInt(Raw);
+            }
+
+            public void ReadBinary(IBinaryReader reader)
+            {
+                Id = reader.ReadInt("id");
+                Name = reader.ReadString("name");
+
+                Raw = reader.GetRawReader().ReadInt();
+            }
+        }
+
+        private class MyComparer : IEqualityComparer<IBinaryObject>
+        {
+            public bool Equals(IBinaryObject x, IBinaryObject y)
+            {
+                return true;
+            }
+
+            public int GetHashCode(IBinaryObject obj)
+            {
+                return 0;
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
index ad5358d..1ebe906 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/IO/BinaryStreamsTest.cs
@@ -94,6 +94,12 @@ namespace Apache.Ignite.Core.Tests.Binary.IO
 
             stream.Write(bytes, 2);
             Assert.AreEqual(2, stream.Position);
+
+            var proc = new SumStreamProcessor();
+            Assert.AreEqual(0, stream.Apply(proc, 0));
+            Assert.AreEqual(1, stream.Apply(proc, 1));
+            Assert.AreEqual(3, stream.Apply(proc, 2));
+
             flush();
 
             seek();
@@ -147,5 +153,18 @@ namespace Apache.Ignite.Core.Tests.Binary.IO
             check(() => stream.WriteShort(4), () => stream.ReadShort(), (short)4);
             check(() => stream.WriteShortArray(new short[] {4}), () => stream.ReadShortArray(1), new short[] {4});
         }
+
+        private class SumStreamProcessor : IBinaryStreamProcessor<int, int>
+        {
+            public unsafe int Invoke(byte* data, int arg)
+            {
+                int res = 0;
+
+                for (var i = 0; i < arg; i++)
+                    res += *(data + i);
+
+                return res;
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 9d55160..fb8725c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -447,6 +447,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 
             Assert.AreEqual(x.Name, y.Name);
             Assert.AreEqual(x.FieldTypeName, y.FieldTypeName);
+            Assert.AreEqual(x.IsKeyField, y.IsKeyField);
         }
 
         /// <summary>
@@ -528,7 +529,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                         Fields = new[]
                         {
                             new QueryField("length", typeof(int)), 
-                            new QueryField("name", typeof(string)), 
+                            new QueryField("name", typeof(string)) {IsKeyField = true}, 
                             new QueryField("location", typeof(string)),
                         },
                         Aliases = new [] {new QueryAlias("length", "len") },
@@ -624,7 +625,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                         {
                             new QueryField("length", typeof(int)), 
                             new QueryField("name", typeof(string)), 
-                            new QueryField("location", typeof(string)),
+                            new QueryField("location", typeof(string)) {IsKeyField = true}
                         },
                         Aliases = new [] {new QueryAlias("length", "len") },
                         Indexes = new[]

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
new file mode 100644
index 0000000..c460252
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheDmlQueriesTest.cs
@@ -0,0 +1,296 @@
+\ufeff/*
+ * 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.
+ */
+
+// ReSharper disable UnusedAutoPropertyAccessor.Local
+namespace Apache.Ignite.Core.Tests.Cache.Query
+{
+    using System.Linq;
+    using Apache.Ignite.Core.Binary;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Binary;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests Data Manipulation Language queries.
+    /// </summary>
+    public class CacheDmlQueriesTest
+    {
+        /// <summary>
+        /// Sets up test fixture.
+        /// </summary>
+        [TestFixtureSetUp]
+        public void FixtureSetUp()
+        {
+            var cfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                BinaryConfiguration = new BinaryConfiguration(typeof(Foo))
+                {
+                    TypeConfigurations =
+                    {
+                        new BinaryTypeConfiguration(typeof(Key))
+                        {
+                            EqualityComparer = new BinaryArrayEqualityComparer()
+                        },
+                        new BinaryTypeConfiguration(typeof(Key2))
+                        {
+                            EqualityComparer = new BinaryFieldEqualityComparer("Hi", "Lo")
+                        }
+                    }
+                }
+            };
+
+            Ignition.Start(cfg);
+        }
+
+        /// <summary>
+        /// Tears down test fixture.
+        /// </summary>
+        [TestFixtureTearDown]
+        public void FixtureTearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests primitive key.
+        /// </summary>
+        [Test]
+        public void TestPrimitiveKey()
+        {
+            var cfg = new CacheConfiguration("primitive_key", new QueryEntity(typeof(int), typeof(Foo)));
+            var cache = Ignition.GetIgnite().CreateCache<int, Foo>(cfg);
+
+            // Test insert.
+            var res = cache.QueryFields(new SqlFieldsQuery("insert into foo(_key, id, name) " +
+                                                           "values (?, ?, ?), (?, ?, ?)",
+                1, 2, "John", 3, 4, "Mary")).GetAll();
+
+            Assert.AreEqual(1, res.Count);
+            Assert.AreEqual(1, res[0].Count);
+            Assert.AreEqual(2, res[0][0]);  // 2 affected rows
+
+            var foos = cache.OrderBy(x => x.Key).ToArray();
+
+            Assert.AreEqual(2, foos.Length);
+
+            Assert.AreEqual(1, foos[0].Key);
+            Assert.AreEqual(2, foos[0].Value.Id);
+            Assert.AreEqual("John", foos[0].Value.Name);
+
+            Assert.AreEqual(3, foos[1].Key);
+            Assert.AreEqual(4, foos[1].Value.Id);
+            Assert.AreEqual("Mary", foos[1].Value.Name);
+
+            // Test key existence.
+            Assert.IsTrue(cache.ContainsKey(1));
+            Assert.IsTrue(cache.ContainsKey(3));
+        }
+
+        /// <summary>
+        /// Tests composite key (which requires QueryField.IsKeyField).
+        /// </summary>
+        [Test]
+        public void TestCompositeKeyArrayEquality()
+        {
+            var cfg = new CacheConfiguration("composite_key_arr", new QueryEntity(typeof(Key), typeof(Foo)));
+            var cache = Ignition.GetIgnite().CreateCache<Key, Foo>(cfg);
+
+            // Test insert.
+            var res = cache.QueryFields(new SqlFieldsQuery("insert into foo(hi, lo, id, name) " +
+                                               "values (1, 2, 3, 'John'), (4, 5, 6, 'Mary')")).GetAll();
+
+            Assert.AreEqual(1, res.Count);
+            Assert.AreEqual(1, res[0].Count);
+            Assert.AreEqual(2, res[0][0]);  // 2 affected rows
+
+            var foos = cache.OrderBy(x => x.Key.Lo).ToArray();
+
+            Assert.AreEqual(2, foos.Length);
+
+            Assert.AreEqual(1, foos[0].Key.Hi);
+            Assert.AreEqual(2, foos[0].Key.Lo);
+            Assert.AreEqual(3, foos[0].Value.Id);
+            Assert.AreEqual("John", foos[0].Value.Name);
+
+            Assert.AreEqual(4, foos[1].Key.Hi);
+            Assert.AreEqual(5, foos[1].Key.Lo);
+            Assert.AreEqual(6, foos[1].Value.Id);
+            Assert.AreEqual("Mary", foos[1].Value.Name);
+
+            // Existence tests check that hash codes are consistent.
+            var binary = cache.Ignite.GetBinary();
+            var binCache = cache.WithKeepBinary<IBinaryObject, IBinaryObject>();
+
+            Assert.IsTrue(cache.ContainsKey(new Key(2, 1)));
+            Assert.IsTrue(cache.ContainsKey(foos[0].Key));
+            Assert.IsTrue(binCache.ContainsKey(
+                binary.GetBuilder(typeof(Key)).SetField("hi", 1).SetField("lo", 2).Build()));
+
+            Assert.IsTrue(cache.ContainsKey(new Key(5, 4)));
+            Assert.IsTrue(cache.ContainsKey(foos[1].Key));
+            Assert.IsTrue(binCache.ContainsKey(
+                binary.GetBuilder(typeof(Key)).SetField("hi", 4).SetField("lo", 5).Build()));
+        }
+
+        /// <summary>
+        /// Tests composite key (which requires QueryField.IsKeyField).
+        /// </summary>
+        [Test]
+        public void TestCompositeKeyFieldEquality()
+        {
+            var cfg = new CacheConfiguration("composite_key_fld", new QueryEntity(typeof(Key2), typeof(Foo)));
+            var cache = Ignition.GetIgnite().CreateCache<Key2, Foo>(cfg);
+
+            // Test insert.
+            var res = cache.QueryFields(new SqlFieldsQuery("insert into foo(hi, lo, str, id, name) " +
+                                               "values (1, 2, 'Foo', 3, 'John'), (4, 5, 'Bar', 6, 'Mary')")).GetAll();
+
+            Assert.AreEqual(1, res.Count);
+            Assert.AreEqual(1, res[0].Count);
+            Assert.AreEqual(2, res[0][0]);  // 2 affected rows
+
+            var foos = cache.OrderBy(x => x.Key.Lo).ToArray();
+
+            Assert.AreEqual(2, foos.Length);
+
+            Assert.AreEqual(1, foos[0].Key.Hi);
+            Assert.AreEqual(2, foos[0].Key.Lo);
+            Assert.AreEqual("Foo", foos[0].Key.Str);
+            Assert.AreEqual(3, foos[0].Value.Id);
+            Assert.AreEqual("John", foos[0].Value.Name);
+
+            Assert.AreEqual(4, foos[1].Key.Hi);
+            Assert.AreEqual(5, foos[1].Key.Lo);
+            Assert.AreEqual("Bar", foos[1].Key.Str);
+            Assert.AreEqual(6, foos[1].Value.Id);
+            Assert.AreEqual("Mary", foos[1].Value.Name);
+
+            // Existence tests check that hash codes are consistent.
+            Assert.IsTrue(cache.ContainsKey(new Key2(2, 1, "Foo")));
+            Assert.IsTrue(cache.ContainsKey(foos[0].Key));
+
+            Assert.IsTrue(cache.ContainsKey(new Key2(5, 4, "Bar")));
+            Assert.IsTrue(cache.ContainsKey(foos[1].Key));
+        }
+
+        /// <summary>
+        /// Tests the composite key without IsKeyField configuration.
+        /// </summary>
+        [Test]
+        public void TestInvalidCompositeKey()
+        {
+            var cfg = new CacheConfiguration("invalid_composite_key", new QueryEntity
+            {
+                KeyTypeName = "Key",
+                ValueTypeName = "Foo",
+                Fields = new[]
+                {
+                    new QueryField("Lo", typeof(int)),
+                    new QueryField("Hi", typeof(int)),
+                    new QueryField("Id", typeof(int)),
+                    new QueryField("Name", typeof(string))
+                }
+            });
+
+            var cache = Ignition.GetIgnite().CreateCache<Key, Foo>(cfg);
+
+            var ex = Assert.Throws<IgniteException>(
+                () => cache.QueryFields(new SqlFieldsQuery("insert into foo(lo, hi, id, name) " +
+                                                           "values (1, 2, 3, 'John'), (4, 5, 6, 'Mary')")));
+
+            Assert.AreEqual("Ownership flag not set for binary property. Have you set 'keyFields' " +
+                            "property of QueryEntity in programmatic or XML configuration?", ex.Message);
+        }
+
+        /// <summary>
+        /// Tests DML with pure binary cache mode, without classes.
+        /// </summary>
+        [Test]
+        public void TestBinaryMode()
+        {
+            var cfg = new CacheConfiguration("binary_only", new QueryEntity
+            {
+                KeyTypeName = "CarKey",
+                ValueTypeName = "Car",
+                Fields = new[]
+                {
+                    new QueryField("VIN", typeof(string)) {IsKeyField = true},
+                    new QueryField("Id", typeof(int)) {IsKeyField = true},
+                    new QueryField("Make", typeof(string)),
+                    new QueryField("Year", typeof(int))
+                }
+            });
+
+            var cache = Ignition.GetIgnite().CreateCache<object, object>(cfg)
+                .WithKeepBinary<IBinaryObject, IBinaryObject>();
+
+            var res = cache.QueryFields(new SqlFieldsQuery("insert into car(VIN, Id, Make, Year) " +
+                                                           "values ('DLRDMC', 88, 'DeLorean', 1982)")).GetAll();
+
+            Assert.AreEqual(1, res.Count);
+            Assert.AreEqual(1, res[0].Count);
+            Assert.AreEqual(1, res[0][0]);
+
+            var car = cache.Single();
+            Assert.AreEqual("CarKey", car.Key.GetBinaryType().TypeName);
+            Assert.AreEqual("Car", car.Value.GetBinaryType().TypeName);
+        }
+
+        /// <summary>
+        /// Key.
+        /// </summary>
+        private struct Key
+        {
+            public Key(int lo, int hi) : this()
+            {
+                Lo = lo;
+                Hi = hi;
+            }
+
+            [QuerySqlField] public int Lo { get; private set; }
+            [QuerySqlField] public int Hi { get; private set; }
+        }
+
+        /// <summary>
+        /// Key.
+        /// </summary>
+        private struct Key2
+        {
+            public Key2(int lo, int hi, string str) : this()
+            {
+                Lo = lo;
+                Hi = hi;
+                Str = str;
+            }
+
+            [QuerySqlField] public int Lo { get; private set; }
+            [QuerySqlField] public int Hi { get; private set; }
+            [QuerySqlField] public string Str { get; private set; }
+        }
+
+        /// <summary>
+        /// Value.
+        /// </summary>
+        private class Foo
+        {
+            [QuerySqlField] public int Id { get; set; }
+            [QuerySqlField] public string Name { get; set; }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
index 55b8dcf..26e04a9 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs
@@ -45,6 +45,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Discovery.Tcp;
     using Apache.Ignite.Core.Discovery.Tcp.Multicast;
     using Apache.Ignite.Core.Events;
+    using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Lifecycle;
     using Apache.Ignite.Core.Log;
@@ -72,6 +73,14 @@ namespace Apache.Ignite.Core.Tests
                                 <types>
                                     <string>Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+FooClass, Apache.Ignite.Core.Tests</string>
                                 </types>
+                                <typeConfigurations>
+                                    <binaryTypeConfiguration affinityKeyFieldName='affKeyFieldName' isEnum='true' keepDeserialized='True' typeName='typeName'>
+                                        <equalityComparer type='BinaryArrayEqualityComparer' />
+                                        <idMapper type='Apache.Ignite.Core.Tests.Binary.IdMapper, Apache.Ignite.Core.Tests' />
+                                        <nameMapper type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+NameMapper, Apache.Ignite.Core.Tests' />
+                                        <serializer type='Apache.Ignite.Core.Tests.IgniteConfigurationSerializerTest+TestSerializer, Apache.Ignite.Core.Tests' />
+                                    </binaryTypeConfiguration>
+                                </typeConfigurations>
                             </binaryConfiguration>
                             <discoverySpi type='TcpDiscoverySpi' joinTimeout='0:1:0' localAddress='192.168.1.1' localPort='6655'>
                                 <ipFinder type='TcpDiscoveryMulticastIpFinder' addressRequestAttempts='7' />
@@ -86,7 +95,7 @@ namespace Apache.Ignite.Core.Tests
                                     <queryEntities>    
                                         <queryEntity keyType='System.Int32' valueType='System.String'>    
                                             <fields>
-                                                <queryField name='length' fieldType='System.Int32' />
+                                                <queryField name='length' fieldType='System.Int32' isKeyField='true' />
                                             </fields>
                                             <aliases>
                                                 <queryAlias fullName='somefield.field' alias='shortField' />
@@ -159,6 +168,7 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(typeof(string), queryEntity.ValueType);
             Assert.AreEqual("length", queryEntity.Fields.Single().Name);
             Assert.AreEqual(typeof(int), queryEntity.Fields.Single().FieldType);
+            Assert.IsTrue(queryEntity.Fields.Single().IsKeyField);
             Assert.AreEqual("somefield.field", queryEntity.Aliases.Single().FullName);
             Assert.AreEqual("shortField", queryEntity.Aliases.Single().Alias);
             Assert.AreEqual(QueryIndexType.Geospatial, queryEntity.Indexes.Single().IndexType);
@@ -214,6 +224,16 @@ namespace Apache.Ignite.Core.Tests
             Assert.AreEqual(25, swap.MaximumWriteQueueSize);
             Assert.AreEqual(36, swap.ReadStripesNumber);
             Assert.AreEqual(47, swap.WriteBufferSize);
+
+            var binType = cfg.BinaryConfiguration.TypeConfigurations.Single();
+            Assert.AreEqual("typeName", binType.TypeName);
+            Assert.AreEqual("affKeyFieldName", binType.AffinityKeyFieldName);
+            Assert.IsTrue(binType.IsEnum);
+            Assert.AreEqual(true, binType.KeepDeserialized);
+            Assert.IsInstanceOf<BinaryArrayEqualityComparer>(binType.EqualityComparer);
+            Assert.IsInstanceOf<IdMapper>(binType.IdMapper);
+            Assert.IsInstanceOf<NameMapper>(binType.NameMapper);
+            Assert.IsInstanceOf<TestSerializer>(binType.Serializer);
         }
 
         /// <summary>
@@ -334,12 +354,12 @@ namespace Apache.Ignite.Core.Tests
             };
 
             Assert.AreEqual(FixLineEndings(@"<?xml version=""1.0"" encoding=""utf-16""?>
-<igniteConfiguration gridName=""myGrid"" clientMode=""true"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
+<igniteConfiguration clientMode=""true"" gridName=""myGrid"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
   <cacheConfiguration>
-    <cacheConfiguration name=""myCache"" cacheMode=""Replicated"">
+    <cacheConfiguration cacheMode=""Replicated"" name=""myCache"">
       <queryEntities>
-        <queryEntity valueTypeName=""java.lang.Integer"" valueType=""System.Int32"" />
-        <queryEntity keyTypeName=""java.lang.Integer"" keyType=""System.Int32"" valueTypeName=""java.lang.String"" valueType=""System.String"" />
+        <queryEntity valueType=""System.Int32"" valueTypeName=""java.lang.Integer"" />
+        <queryEntity keyType=""System.Int32"" keyTypeName=""java.lang.Integer"" valueType=""System.String"" valueTypeName=""java.lang.String"" />
       </queryEntities>
     </cacheConfiguration>
   </cacheConfiguration>
@@ -364,12 +384,12 @@ namespace Apache.Ignite.Core.Tests
             }
 
             Assert.AreEqual(FixLineEndings(@"<?xml version=""1.0"" encoding=""utf-16""?>
-<igCfg gridName=""myGrid"" clientMode=""true"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
+<igCfg clientMode=""true"" gridName=""myGrid"" xmlns=""http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"">
  <cacheConfiguration>
-  <cacheConfiguration name=""myCache"" cacheMode=""Replicated"">
+  <cacheConfiguration cacheMode=""Replicated"" name=""myCache"">
    <queryEntities>
-    <queryEntity valueTypeName=""java.lang.Integer"" valueType=""System.Int32"" />
-    <queryEntity keyTypeName=""java.lang.Integer"" keyType=""System.Int32"" valueTypeName=""java.lang.String"" valueType=""System.String"" />
+    <queryEntity valueType=""System.Int32"" valueTypeName=""java.lang.Integer"" />
+    <queryEntity keyType=""System.Int32"" keyTypeName=""java.lang.Integer"" valueType=""System.String"" valueTypeName=""java.lang.String"" />
    </queryEntities>
   </cacheConfiguration>
  </cacheConfiguration>
@@ -547,7 +567,8 @@ namespace Apache.Ignite.Core.Tests
                             TypeName = "typeName",
                             IdMapper = new IdMapper(),
                             NameMapper = new NameMapper(),
-                            Serializer = new TestSerializer()
+                            Serializer = new TestSerializer(),
+                            EqualityComparer = new BinaryArrayEqualityComparer()
                         },
                         new BinaryTypeConfiguration
                         {
@@ -555,7 +576,8 @@ namespace Apache.Ignite.Core.Tests
                             KeepDeserialized = false,
                             AffinityKeyFieldName = "affKeyFieldName",
                             TypeName = "typeName2",
-                            Serializer = new BinaryReflectiveSerializer()
+                            Serializer = new BinaryReflectiveSerializer(),
+                            EqualityComparer = new BinaryFieldEqualityComparer()
                         }
                     },
                     Types = new[] {typeof (string).FullName},
@@ -595,7 +617,7 @@ namespace Apache.Ignite.Core.Tests
                             {
                                 Fields = new[]
                                 {
-                                    new QueryField("field", typeof (int))
+                                    new QueryField("field", typeof (int)) { IsKeyField = true }
                                 },
                                 Indexes = new[]
                                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
index 2e39b9b..86ece98 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs
@@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Discovery.Tcp.Static;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.SwapSpace.File;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
@@ -181,6 +182,18 @@ namespace Apache.Ignite.Core.Tests
                 Assert.AreEqual(swap.MaximumWriteQueueSize, resSwap.MaximumWriteQueueSize);
                 Assert.AreEqual(swap.ReadStripesNumber, resSwap.ReadStripesNumber);
                 Assert.AreEqual(swap.WriteBufferSize, resSwap.WriteBufferSize);
+
+                var binCfg = cfg.BinaryConfiguration;
+                Assert.IsFalse(binCfg.CompactFooter);
+
+                var typ = binCfg.TypeConfigurations.Single();
+                Assert.AreEqual("myType", typ.TypeName);
+                Assert.IsTrue(typ.IsEnum);
+                Assert.AreEqual("affKey", typ.AffinityKeyFieldName);
+                Assert.AreEqual(false, typ.KeepDeserialized);
+
+                CollectionAssert.AreEqual(new[] {"fld1", "fld2"}, 
+                    ((BinaryFieldEqualityComparer)typ.EqualityComparer).FieldNames);
             }
         }
 
@@ -513,6 +526,21 @@ namespace Apache.Ignite.Core.Tests
                     WriteBufferSize = 9,
                     BaseDirectory = Path.GetTempPath(),
                     MaximumSparsity = 11.22f
+                },
+                BinaryConfiguration = new BinaryConfiguration
+                {
+                    CompactFooter = false,
+                    TypeConfigurations = new[]
+                    {
+                        new BinaryTypeConfiguration
+                        {
+                            TypeName = "myType",
+                            IsEnum = true,
+                            AffinityKeyFieldName = "affKey",
+                            KeepDeserialized = false,
+                            EqualityComparer = new BinaryFieldEqualityComparer("fld1", "fld2")
+                        }
+                    }
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index a80dfc0..42ccdd4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -90,6 +90,8 @@
     <Reference Include="System.Xml" />
   </ItemGroup>
   <ItemGroup>
+    <Compile Include="Binary\BinaryArrayEqualityComparer.cs" />
+    <Compile Include="Impl\Binary\BinaryFieldEqualityComparer.cs" />
     <Compile Include="Binary\BinaryReflectiveSerializer.cs" />
     <Compile Include="Common\JavaException.cs" />
     <Compile Include="DataStructures\Configuration\Package-Info.cs" />
@@ -97,8 +99,10 @@
     <Compile Include="Discovery\Tcp\Multicast\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Package-Info.cs" />
     <Compile Include="Discovery\Tcp\Static\Package-Info.cs" />
+    <Compile Include="Impl\Binary\BinaryEqualityComparerSerializer.cs" />
     <Compile Include="Impl\Binary\BinaryProcessor.cs" />
     <Compile Include="Impl\Binary\BinaryReflectiveSerializerInternal.cs" />
+    <Compile Include="Impl\Binary\IBinaryEqualityComparer.cs" />
     <Compile Include="Impl\Binary\IBinarySerializerInternal.cs" />
     <Compile Include="Binary\Package-Info.cs" />
     <Compile Include="Cache\Affinity\AffinityKey.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
new file mode 100644
index 0000000..09f7f0f
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryArrayEqualityComparer.cs
@@ -0,0 +1,149 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Binary
+{
+    using System;
+    using System.Collections.Generic;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Binary.IO;
+
+    /// <summary>
+    /// Compares binary object equality using underlying byte array.
+    /// </summary>
+    public class BinaryArrayEqualityComparer : IEqualityComparer<IBinaryObject>, IBinaryEqualityComparer,
+        IBinaryStreamProcessor<KeyValuePair<int,int>, int>
+    {
+        /// <summary>
+        /// Determines whether the specified objects are equal.
+        /// </summary>
+        /// <param name="x">The first object to compare.</param>
+        /// <param name="y">The second object to compare.</param>
+        /// <returns>
+        /// true if the specified objects are equal; otherwise, false.
+        /// </returns>
+        public bool Equals(IBinaryObject x, IBinaryObject y)
+        {
+            if (x == null)
+                return y == null;
+
+            if (y == null)
+                return false;
+
+            if (ReferenceEquals(x, y))
+                return true;
+
+            var binx = GetBinaryObject(x);
+            var biny = GetBinaryObject(y);
+
+            var lenx = GetDataLength(binx);
+            var leny = GetDataLength(biny);
+
+            if (lenx != leny)
+                return false;
+
+            var startx = GetDataStart(binx);
+            var starty = GetDataStart(biny);
+
+            var arrx = binx.Data;
+            var arry = biny.Data;
+
+            for (var i = 0; i < lenx; i++)
+            {
+                if (arrx[i + startx] != arry[i + starty])
+                    return false;
+            }
+
+            return true;
+        }
+
+        /// <summary>
+        /// Returns a hash code for this instance.
+        /// </summary>
+        /// <param name="obj">The object.</param>
+        /// <returns>
+        /// A hash code for this instance, suitable for use in hashing algorithms and data structures like a hash table. 
+        /// </returns>
+        public int GetHashCode(IBinaryObject obj)
+        {
+            if (obj == null)
+                return 0;
+
+            var binObj = GetBinaryObject(obj);
+
+            var arg = new KeyValuePair<int, int>(GetDataStart(binObj), GetDataLength(binObj));
+
+            return new BinaryHeapStream(binObj.Data).Apply(this, arg);
+        }
+
+        /** <inheritdoc /> */
+        int IBinaryEqualityComparer.GetHashCode(IBinaryStream stream, int startPos, int length, 
+            BinaryObjectSchemaHolder schema, int schemaId, Marshaller marshaller, IBinaryTypeDescriptor desc)
+        {
+            Debug.Assert(stream != null);
+            Debug.Assert(startPos >= 0);
+            Debug.Assert(length >= 0);
+
+            var arg = new KeyValuePair<int, int>(startPos, length);
+
+            return stream.Apply(this, arg);
+        }
+
+        /** <inheritdoc /> */
+        unsafe int IBinaryStreamProcessor<KeyValuePair<int, int>, int>.Invoke(byte* data, KeyValuePair<int, int> arg)
+        {
+            var hash = 1;
+            var ptr = data + arg.Key;
+
+            for (var i = 0; i < arg.Value; i++)
+                hash = 31 * hash + *(ptr + i);
+
+            return hash;
+        }
+
+        /// <summary>
+        /// Casts to <see cref="BinaryObject"/> or throws an error.
+        /// </summary>
+        private static BinaryObject GetBinaryObject(IBinaryObject obj)
+        {
+            var binObj = obj as BinaryObject;
+
+            if (binObj != null)
+                return binObj;
+
+            throw new NotSupportedException(string.Format("{0} of type {1} is not supported.",
+                typeof(IBinaryObject), obj.GetType()));
+        }
+
+        /// <summary>
+        /// Gets the non-raw data length.
+        /// </summary>
+        private static int GetDataLength(BinaryObject binObj)
+        {
+            return binObj.Header.FooterStartOffset - BinaryObjectHeader.Size;
+        }
+
+        /// <summary>
+        /// Gets the data starting position.
+        /// </summary>
+        private static int GetDataStart(BinaryObject binObj)
+        {
+            return binObj.Offset + BinaryObjectHeader.Size;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
index 51df907..c738f28 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryConfiguration.cs
@@ -141,5 +141,29 @@ namespace Apache.Ignite.Core.Binary
         {
             get { return _compactFooter; }
         }
+
+        /// <summary>
+        /// Merges other config into this.
+        /// </summary>
+        internal void MergeTypes(BinaryConfiguration localConfig)
+        {
+            if (TypeConfigurations == null)
+            {
+                TypeConfigurations = localConfig.TypeConfigurations;
+            }
+            else if (localConfig.TypeConfigurations != null)
+            {
+                // Both configs are present.
+                // Local configuration is more complete and takes preference when it exists for a given type.
+                var localTypeNames = new HashSet<string>(localConfig.TypeConfigurations.Select(x => x.TypeName), 
+                    StringComparer.OrdinalIgnoreCase);
+
+                var configs = new List<BinaryTypeConfiguration>(localConfig.TypeConfigurations);
+
+                configs.AddRange(TypeConfigurations.Where(x=>!localTypeNames.Contains(x.TypeName)));
+
+                TypeConfigurations = configs;
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
index c36b9fd..722197c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Binary/BinaryTypeConfiguration.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Binary
 {
     using System;
+    using System.Collections.Generic;
     using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
@@ -69,6 +70,7 @@ namespace Apache.Ignite.Core.Binary
             TypeName = cfg.TypeName;
             KeepDeserialized = cfg.KeepDeserialized;
             IsEnum = cfg.IsEnum;
+            EqualityComparer = cfg.EqualityComparer;
         }
 
         /// <summary>
@@ -113,6 +115,18 @@ namespace Apache.Ignite.Core.Binary
         public bool IsEnum { get; set; }
 
         /// <summary>
+        /// Gets or sets the equality comparer to compute hash codes and compare objects
+        /// in <see cref="IBinaryObject"/> form.
+        /// This comparer is important only for types that are used as cache keys.
+        /// <para />
+        /// Null means legacy behavior: hash code is computed by calling <see cref="object.GetHashCode"/>, equality is
+        /// computed by comparing bytes in serialized (binary) form.
+        /// <para />
+        /// Only predefined implementations are supported: <see cref="BinaryArrayEqualityComparer"/>.
+        /// </summary>
+        public IEqualityComparer<IBinaryObject> EqualityComparer { get; set; }
+
+        /// <summary>
         /// Returns a string that represents the current object.
         /// </summary>
         /// <returns>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index adfe9e1..ff9af37 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -178,8 +178,11 @@ namespace Apache.Ignite.Core.Cache.Configuration
             ValueTypeName = reader.ReadString();
 
             var count = reader.ReadInt();
-            Fields = count == 0 ? null : Enumerable.Range(0, count).Select(x =>
-                    new QueryField(reader.ReadString(), reader.ReadString())).ToList();
+            Fields = count == 0
+                ? null
+                : Enumerable.Range(0, count).Select(x =>
+                    new QueryField(reader.ReadString(), reader.ReadString()) {IsKeyField = reader.ReadBoolean()})
+                    .ToList();
 
             count = reader.ReadInt();
             Aliases = count == 0 ? null : Enumerable.Range(0, count)
@@ -205,6 +208,7 @@ namespace Apache.Ignite.Core.Cache.Configuration
                 {
                     writer.WriteString(field.Name);
                     writer.WriteString(field.FieldTypeName);
+                    writer.WriteBoolean(field.IsKeyField);
                 }
             }
             else
@@ -264,16 +268,19 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// <summary>
         /// Rescans the attributes in <see cref="KeyType"/> and <see cref="ValueType"/>.
         /// </summary>
-        private void RescanAttributes(params Type[] types)
+        private void RescanAttributes(Type keyType, Type valType)
         {
-            if (types.Length == 0 || types.All(t => t == null))
+            if (keyType == null && valType == null)
                 return;
 
             var fields = new List<QueryField>();
             var indexes = new List<QueryIndexEx>();
 
-            foreach (var type in types.Where(t => t != null))
-                ScanAttributes(type, fields, indexes, null, new HashSet<Type>());
+            if (keyType != null)
+                ScanAttributes(keyType, fields, indexes, null, new HashSet<Type>(), true);
+
+            if (valType != null)
+                ScanAttributes(valType, fields, indexes, null, new HashSet<Type>(), false);
 
             if (fields.Any())
                 Fields = fields;
@@ -308,15 +315,17 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
-        /// Scans specified type for occurences of <see cref="QuerySqlFieldAttribute"/>.
+        /// Scans specified type for occurences of <see cref="QuerySqlFieldAttribute" />.
         /// </summary>
         /// <param name="type">The type.</param>
         /// <param name="fields">The fields.</param>
         /// <param name="indexes">The indexes.</param>
         /// <param name="parentPropName">Name of the parent property.</param>
         /// <param name="visitedTypes">The visited types.</param>
+        /// <param name="isKey">Whether this is a key type.</param>
+        /// <exception cref="System.InvalidOperationException">Recursive Query Field definition detected:  + type</exception>
         private static void ScanAttributes(Type type, List<QueryField> fields, List<QueryIndexEx> indexes, 
-            string parentPropName, ISet<Type> visitedTypes)
+            string parentPropName, ISet<Type> visitedTypes, bool isKey)
         {
             Debug.Assert(type != null);
             Debug.Assert(fields != null);
@@ -344,9 +353,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
                     if (parentPropName != null)
                         columnName = parentPropName + "." + columnName;
 
-                    fields.Add(new QueryField(columnName, memberInfo.Value));
+                    fields.Add(new QueryField(columnName, memberInfo.Value) {IsKeyField = isKey});
 
-                    ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes);
+                    ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes, isKey);
                 }
 
                 foreach (var attr in customAttributes.OfType<QueryTextFieldAttribute>())
@@ -359,9 +368,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
                     if (parentPropName != null)
                         columnName = parentPropName + "." + columnName;
 
-                    fields.Add(new QueryField(columnName, memberInfo.Value));
+                    fields.Add(new QueryField(columnName, memberInfo.Value) {IsKeyField = isKey});
 
-                    ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes);
+                    ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes, isKey);
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b7908d7a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
index 12028e2..c33aa57 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
@@ -109,6 +109,12 @@ namespace Apache.Ignite.Core.Cache.Configuration
         }
 
         /// <summary>
+        /// Gets or sets a value indicating whether this field belongs to the cache key.
+        /// Proper value here is required for SQL DML queries which create/modify cache keys.
+        /// </summary>
+        public bool IsKeyField { get; set; }
+
+        /// <summary>
         /// Validates this instance and outputs information to the log, if necessary.
         /// </summary>
         internal void Validate(ILogger log, string logInfo)


[09/50] [abbrv] ignite git commit: IGNITE-4519 updating versions for gce and jcloud

Posted by sb...@apache.org.
IGNITE-4519 updating versions for gce and jcloud


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 2774d879a72b0eeced862cc9a3fbd5d9c5ff2d72
Parents: 6c1cd16
Author: chandresh.pancholi <ch...@arvindinternet.com>
Authored: Thu Jan 5 02:31:13 2017 +0530
Committer: chandresh.pancholi <ch...@arvindinternet.com>
Committed: Thu Jan 5 02:31:13 2017 +0530

----------------------------------------------------------------------
 modules/cloud/pom.xml | 6 +++---
 modules/gce/pom.xml   | 4 ++--
 2 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2774d879/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 5ac1990..8d806f4 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -33,7 +33,7 @@
     <url>http://ignite.apache.org</url>
 
     <properties>
-        <jcloud.version>1.9.0</jcloud.version>
+        <jcloud.version>2.0.0</jcloud.version>
     </properties>
 
     <dependencies>
@@ -52,13 +52,13 @@
         <dependency>
             <groupId>org.apache.jclouds.labs</groupId>
             <artifactId>google-compute-engine</artifactId>
-            <version>${jcloud.version}</version>
+            <version>1.9.3</version>
         </dependency>
 
         <dependency>
             <groupId>org.apache.jclouds.labs</groupId>
             <artifactId>docker</artifactId>
-            <version>${jcloud.version}</version>
+            <version>1.9.3</version>
         </dependency>
 
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/2774d879/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index b235d82..89f9a8b 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -44,13 +44,13 @@
         <dependency>
             <groupId>com.google.api-client</groupId>
             <artifactId>google-api-client</artifactId>
-            <version>1.19.1</version>
+            <version>1.22.0</version>
         </dependency>
 
         <dependency>
             <groupId>com.google.apis</groupId>
             <artifactId>google-api-services-storage</artifactId>
-            <version>v1-rev32-1.20.0</version>
+            <version>v1-rev92-1.22.0</version>
         </dependency>
 
         <dependency>


[20/50] [abbrv] ignite git commit: IGNITE-3885 .NET: Add link to Ignite.NET development page in readme.md

Posted by sb...@apache.org.
IGNITE-3885 .NET: Add link to Ignite.NET development page in readme.md


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 58188e81b262c26f93b29f3facd3e77f4325e359
Parents: c4d67d2
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 17:32:57 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 17:32:57 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/README.md | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/58188e81/modules/platforms/dotnet/README.md
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/README.md b/modules/platforms/dotnet/README.md
index 528a219..cbdc7c9 100644
--- a/modules/platforms/dotnet/README.md
+++ b/modules/platforms/dotnet/README.md
@@ -147,4 +147,8 @@ You can find the full Apache Ignite documentation here: [Full documentation][doc
 
 [apache-homepage]: https://ignite.apache.org/
 [getting-started]: https://apacheignite-net.readme.io/docs/getting-started-2
-[docs]: https://apacheignite-net.readme.io/docs
\ No newline at end of file
+[docs]: https://apacheignite-net.readme.io/docs
+
+## Contribute to Ignite.NET
+
+See [Ignite.NET Development](https://cwiki.apache.org/confluence/display/IGNITE/Ignite.NET+Development) on wiki.
\ No newline at end of file


[14/50] [abbrv] ignite git commit: .NET: Fix non-ascii chars in AssemblyInfo

Posted by sb...@apache.org.
.NET: Fix non-ascii chars in AssemblyInfo


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

Branch: refs/heads/ignite-gg-11810-1
Commit: d2e6007b905b6c19cd87786a039229156d10c013
Parents: f406887
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Mon Jan 9 12:40:42 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 9 12:40:42 2017 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs   | 2 +-
 .../dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs         | 2 +-
 .../dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs     | 2 +-
 3 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
index f5fa618..1bca0e8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
index d72c9db..0926a46 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d2e6007b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
index cc833ea..1fc6c59 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Properties/AssemblyInfo.cs
@@ -1,4 +1,4 @@
-\ufeff\ufeff/*
+\ufeff/*
 * 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.


[50/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-3477' into ignite-gg-11810-1

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-3477' into ignite-gg-11810-1

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/io/BPlusMetaIO.java


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 8f005c3084a4ee1d6e1b2a0bba275e781edfd7ab
Parents: fcfd62a 7db65dd
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jan 17 16:59:26 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jan 17 16:59:26 2017 +0300

----------------------------------------------------------------------
 modules/cloud/pom.xml                           |   6 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../ignite/cache/affinity/AffinityKey.java      |   4 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   6 +
 .../configuration/HadoopConfiguration.java      |   2 +-
 .../configuration/IgniteConfiguration.java      |  31 +
 .../org/apache/ignite/events/CacheEvent.java    |   6 +-
 .../ignite/events/CacheQueryReadEvent.java      |   8 +-
 .../apache/ignite/hadoop/HadoopInputSplit.java  |  54 ++
 .../org/apache/ignite/hadoop/HadoopJob.java     |  74 +++
 .../ignite/hadoop/HadoopMapReducePlan.java      |  80 +++
 .../ignite/hadoop/HadoopMapReducePlanner.java   |  40 ++
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |  17 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  28 +
 .../local/LocalIgfsSecondaryFileSystem.java     | 130 +++-
 .../ignite/internal/GridKernalContext.java      |   7 +
 .../ignite/internal/GridKernalContextImpl.java  |  12 +
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  19 +
 .../internal/binary/BinaryEnumObjectImpl.java   |  10 +-
 .../ignite/internal/binary/BinaryMetadata.java  |   5 +-
 .../internal/binary/BinaryObjectExImpl.java     |   8 +-
 .../igfs/common/IgfsHandshakeRequest.java       |  17 -
 .../internal/igfs/common/IgfsMarshaller.java    |   2 -
 .../managers/communication/GridIoManager.java   |   2 +
 .../managers/communication/GridIoPolicy.java    |   3 +
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../processors/cache/CacheInvokeResult.java     |   2 +-
 .../processors/cache/CacheLazyEntry.java        |   4 +-
 .../processors/cache/CacheObjectAdapter.java    |   7 +-
 .../processors/cache/GridCacheAdapter.java      |   5 +-
 .../cache/GridCacheMvccCandidate.java           |   9 +-
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   2 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  38 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 +-
 .../distributed/near/GridNearLockFuture.java    |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |   4 +-
 .../cache/query/GridCacheQueryManager.java      |  13 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +
 .../cache/query/GridCacheSqlQuery.java          |   6 +-
 .../continuous/CacheContinuousQueryEvent.java   |  10 +-
 .../continuous/CacheContinuousQueryManager.java |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  30 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../GridCacheVersionConflictContext.java        |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   5 +-
 .../closure/GridClosureProcessor.java           |   6 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastreamer/DataStreamProcessor.java       |  60 +-
 .../datastreamer/DataStreamerImpl.java          |  37 +-
 .../datastructures/CollocatedSetItemKey.java    |   2 +-
 .../GridCacheAtomicLongValue.java               |   2 +
 .../GridCacheAtomicSequenceImpl.java            |   2 +
 .../GridCacheAtomicSequenceValue.java           |   2 +
 .../GridCacheCountDownLatchValue.java           |   3 +
 .../datastructures/GridCacheSetItemKey.java     |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   4 +-
 .../processors/hadoop/HadoopFileBlock.java      |   1 +
 .../processors/hadoop/HadoopInputSplit.java     |  54 --
 .../internal/processors/hadoop/HadoopJob.java   | 107 ---
 .../internal/processors/hadoop/HadoopJobEx.java | 140 ++++
 .../processors/hadoop/HadoopJobInfo.java        |  54 +-
 .../processors/hadoop/HadoopJobProperty.java    |   9 +-
 .../processors/hadoop/HadoopMapReducePlan.java  |  80 ---
 .../hadoop/HadoopMapReducePlanner.java          |  40 --
 .../processors/hadoop/HadoopTaskContext.java    |   6 +-
 .../processors/hadoop/HadoopTaskInfo.java       |   1 +
 .../hadoop/counter/HadoopCounterWriter.java     |   4 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |  34 +-
 .../processors/igfs/IgfsBaseBlockKey.java       |  42 ++
 .../internal/processors/igfs/IgfsBlockKey.java  |  26 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |  55 ++
 .../processors/igfs/IgfsDataManager.java        |  12 +-
 .../internal/processors/igfs/IgfsImpl.java      |  26 +-
 .../processors/igfs/IgfsIpcHandler.java         |   4 -
 .../processors/igfs/IgfsKernalContextAware.java |  32 -
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |  10 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |  40 --
 .../local/LocalFileSystemBlockKey.java          | 103 +++
 .../local/LocalFileSystemIgfsFile.java          |  10 +-
 .../LocalFileSystemPositionedReadable.java      |  65 ++
 ...fsSecondaryFileSystemPositionedReadable.java |  65 --
 .../internal/processors/job/GridJobWorker.java  |   7 +-
 .../odbc/OdbcQueryExecuteRequest.java           |   6 +-
 .../platform/PlatformNativeException.java       |   3 +-
 .../utils/PlatformConfigurationUtils.java       | 131 +++-
 .../internal/processors/pool/PoolProcessor.java |   5 +
 .../processors/query/GridQueryProcessor.java    |  37 +-
 .../processors/resource/GridResourceIoc.java    |   6 +-
 .../resource/GridResourceProcessor.java         |  31 +-
 .../processors/rest/GridRestResponse.java       |   2 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 159 ++++-
 .../util/tostring/GridToStringBuilder.java      | 656 +++++++++++++++++--
 .../util/tostring/GridToStringInclude.java      |  12 +-
 .../util/tostring/GridToStringThreadLocal.java  |  12 +-
 .../query/VisorQueryScanSubstringFilter.java    |   5 +-
 .../internal/visor/query/VisorQueryUtils.java   |  60 ++
 .../ignite/resources/FileSystemResource.java    |  62 ++
 .../communication/tcp/TcpCommunicationSpi.java  |  20 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   5 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   3 +
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  16 +-
 .../resources/META-INF/classnames.properties    |   5 +-
 .../ignite/cache/store/jdbc/model/Person.java   |   2 +-
 ...mmunicationBalancePairedConnectionsTest.java |  28 +
 .../IgniteCommunicationBalanceTest.java         |  25 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   6 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 --
 ...micMessageRecoveryPairedConnectionsTest.java |  47 ++
 .../AtomicPutAllChangingTopologyTest.java       | 212 ++++++
 .../cache/query/IndexingSpiQuerySelfTest.java   | 199 +++++-
 .../DataStreamProcessorSelfTest.java            | 104 +++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  18 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |  26 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |   5 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |  81 +++
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |   7 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java | 270 ++++++++
 .../tostring/GridToStringBuilderSelfTest.java   |  33 +-
 .../junits/GridTestKernalContext.java           |  12 +-
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   3 +
 .../apache/ignite/stream/flume/IgniteSink.java  |   7 +-
 modules/gce/pom.xml                             |   4 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   6 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  23 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java | 416 ------------
 .../IgniteHadoopWeightedMapReducePlanner.java   |  15 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 118 ++++
 .../planner/HadoopTestRoundRobinMrPlanner.java  |  75 +++
 .../processors/hadoop/HadoopCommonUtils.java    |   1 +
 .../processors/hadoop/HadoopContext.java        |   2 +
 .../processors/hadoop/HadoopExternalSplit.java  |   1 +
 .../processors/hadoop/HadoopProcessor.java      |   4 +-
 .../processors/hadoop/HadoopSplitWrapper.java   |   1 +
 .../HadoopFileSystemCounterWriterDelegate.java  |   4 +-
 .../HadoopIgfsSecondaryFileSystemDelegate.java  |   4 +-
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |  38 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |   4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |  61 +-
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   9 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |  24 +-
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  33 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |   6 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |   4 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   2 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |  15 +-
 .../hadoop/impl/v2/HadoopV2Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  77 ++-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |   4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  32 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 116 ----
 .../planner/HadoopDefaultMapReducePlan.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   4 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  66 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  75 ++-
 .../shuffle/direct/HadoopDirectDataOutput.java  |  14 +
 .../direct/HadoopDirectDataOutputContext.java   |  48 +-
 .../direct/HadoopDirectDataOutputState.java     |  14 +-
 .../shuffle/streams/HadoopDataInStream.java     |  34 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |  18 +
 .../HadoopEmbeddedTaskExecutor.java             |   8 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   6 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   8 +-
 .../external/HadoopExternalTaskExecutor.java    |  16 +-
 .../child/HadoopChildProcessRunner.java         |   4 +-
 .../resources/META-INF/classnames.properties    |   8 +
 .../impl/HadoopAbstractMapReduceTest.java       |   2 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   3 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |   4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 619 -----------------
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   1 +
 .../hadoop/impl/HadoopPlannerMockJob.java       |  10 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |  19 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |  16 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |   4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |   4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  32 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |  75 ---
 .../hadoop/impl/HadoopTestTaskContext.java      |   6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |   6 +-
 .../HadoopWeightedMapReducePlannerTest.java     |   6 +-
 .../client/HadoopClientProtocolSelfTest.java    |   2 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   7 +-
 .../impl/igfs/Hadoop1OverIgfsProxyTest.java     |  67 ++
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   6 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   2 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   2 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   6 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   4 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  32 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   6 +-
 ...condaryFileSystemInitializationSelfTest.java |   4 +-
 .../collections/HadoopAbstractMapTest.java      |   4 +-
 .../streams/HadoopDataStreamSelfTest.java       | 177 ++++-
 .../testsuites/IgniteHadoopTestSuite.java       |   4 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   6 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  18 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   4 +-
 .../org/apache/ignite/logger/jcl/JclLogger.java |   9 +-
 modules/platforms/cpp/binary/Makefile.am        |   2 +-
 .../ignite/impl/binary/binary_type_manager.h    |   2 +-
 .../src/impl/binary/binary_type_manager.cpp     |   6 +-
 .../binary/src/impl/interop/interop_memory.cpp  |   2 +-
 modules/platforms/cpp/common/Makefile.am        |   2 +-
 .../include/ignite/common/dynamic_size_array.h  |   2 +-
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 modules/platforms/cpp/configure.ac              |   2 +-
 modules/platforms/cpp/configure.acrel           |   2 +-
 modules/platforms/cpp/core-test/Makefile.am     |   2 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   8 +-
 .../cpp/core-test/src/reference_test.cpp        |  14 +-
 modules/platforms/cpp/core/Makefile.am          |   2 +-
 .../cache/query/continuous/continuous_query.h   |   8 +-
 .../include/ignite/cache/query/query_cursor.h   |   4 +-
 .../query/continuous/continuous_query_impl.h    |   8 +-
 .../core/include/ignite/impl/handle_registry.h  |   2 +-
 .../cpp/core/include/ignite/impl/operations.h   |   4 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   5 +-
 .../cpp/core/src/impl/handle_registry.cpp       |   2 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |   2 +-
 .../cpp/examples/putget-example/Makefile.am     |   2 +-
 .../cpp/examples/query-example/Makefile.am      |   2 +-
 modules/platforms/cpp/ignite/Makefile.am        |   2 +-
 modules/platforms/cpp/jni/Makefile.am           |   2 +-
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |   1 +
 modules/platforms/cpp/jni/src/java.cpp          |   2 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +-
 .../cpp/odbc-test/config/queries-default.xml    |   1 +
 .../odbc-test/include/sql_test_suite_fixture.h  |   3 +
 .../platforms/cpp/odbc-test/include/test_type.h |  20 +-
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++-
 .../platforms/cpp/odbc-test/src/column_test.cpp | 189 ++++++
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  17 +
 .../cpp/odbc-test/src/sql_types_test.cpp        | 131 ++++
 modules/platforms/cpp/odbc/Makefile.am          |   5 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++-
 .../cpp/odbc/include/ignite/odbc/utility.h      |  26 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |   2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |   2 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 modules/platforms/cpp/odbc/src/column.cpp       |  28 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   9 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 ++
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 +-
 modules/platforms/cpp/odbc/src/log.cpp          |  83 +++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 286 ++++----
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 +++++-
 modules/platforms/cpp/odbc/src/utility.cpp      |  29 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |   7 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.AspNet.csproj                 |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Benchmarks.csproj             |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.csproj             |  10 +
 .../Binary/BinaryBuilderSelfTest.cs             | 159 +++--
 .../BinaryBuilderSelfTestArrayIdentity.cs       |  34 +
 .../Binary/BinaryEqualityComparerTest.cs        | 279 ++++++++
 .../Binary/IO/BinaryStreamsTest.cs              |  19 +
 .../Cache/CacheAbstractTest.cs                  | 627 +-----------------
 .../Cache/CacheAbstractTransactionalTest.cs     | 565 ++++++++++++++++
 .../Cache/CacheConfigurationTest.cs             |   5 +-
 .../Cache/CacheLocalAtomicTest.cs               |   5 -
 .../Cache/CacheLocalTest.cs                     |   6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |   5 -
 .../Cache/CachePartitionedAtomicTest.cs         |   5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   7 +-
 .../Cache/CachePartitionedTest.cs               |   7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |   5 -
 .../Cache/CacheReplicatedTest.cs                |   7 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          | 296 +++++++++
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../IgniteConfigurationSerializerTest.cs        |  46 +-
 .../IgniteConfigurationTest.cs                  |  28 +
 .../Log/ConcurrentMemoryTarget.cs               |  73 +++
 .../Log/NLogLoggerTest.cs                       |   5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   6 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |   5 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   1 -
 .../Apache.Ignite.Core.csproj                   |  21 +-
 .../Binary/BinaryArrayEqualityComparer.cs       | 149 +++++
 .../Binary/BinaryConfiguration.cs               |  24 +
 .../Binary/BinaryTypeConfiguration.cs           |  14 +
 .../Cache/Configuration/QueryEntity.cs          |  33 +-
 .../Cache/Configuration/QueryField.cs           |   6 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  85 ++-
 .../IgniteConfigurationSection.xsd              |  19 +
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |  28 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |  99 +++
 .../Impl/Binary/BinaryFieldEqualityComparer.cs  | 138 ++++
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  21 +-
 .../Impl/Binary/BinaryObject.cs                 |  31 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  62 +-
 .../Impl/Binary/BinaryObjectHeader.cs           |  21 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  22 +
 .../Impl/Binary/BinaryReader.cs                 |  16 +-
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   6 +
 .../Impl/Binary/BinarySystemHandlers.cs         |   6 +-
 .../Impl/Binary/BinaryWriter.cs                 |  11 +-
 .../Impl/Binary/DateTimeHolder.cs               |  35 +-
 .../Impl/Binary/IBinaryEqualityComparer.cs      |  53 ++
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   5 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |   9 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  13 +
 .../Impl/Binary/Io/IBinaryStream.cs             |  11 +-
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    |  36 +
 .../Impl/Binary/Marshaller.cs                   |  22 +-
 .../Impl/Binary/SerializableObjectHolder.cs     |  16 +
 .../Common/IgniteConfigurationXmlSerializer.cs  |   5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |  16 +
 .../Impl/Transactions/TransactionsImpl.cs       |  18 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Transactions/ITransactions.cs               |  19 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.EntityFramework.csproj        |   9 +-
 .../Impl/ArrayDbDataReader.cs                   |   5 +
 .../Impl/DataReaderResult.cs                    |   2 +
 .../Impl/DbCommandInfo.cs                       |  11 +
 .../Impl/DbCommandProxy.cs                      |   1 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Linq.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Log4Net.csproj                |   5 +
 .../IgniteLog4NetLogger.cs                      |  16 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.NLog.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |   3 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   5 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 modules/platforms/dotnet/README.md              |   6 +-
 modules/platforms/dotnet/build.ps1              |   2 +-
 .../Apache.Ignite.Examples.csproj               |   3 +
 .../examples/Apache.Ignite.Examples/App.config  |   2 +
 .../DataStructures/AtomicSequenceExample.cs     |  26 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.ExamplesDll.csproj            |   4 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Services/MapService.cs                      |   2 -
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |  11 +-
 modules/web-console/backend/app/mongo.js        |  19 +
 .../generator/AbstractTransformer.js            |   5 +
 .../modules/configuration/generator/Beans.js    |   4 +
 .../generator/ConfigurationGenerator.js         |  36 +
 .../states/configuration/caches/affinity.jade   |  82 +++
 .../states/configuration/caches/memory.jade     |   4 +-
 .../frontend/views/configuration/caches.jade    |   1 +
 .../yardstick/cache/IgniteIoTestBenchmark.java  |  73 ---
 parent/pom.xml                                  |   2 +-
 389 files changed, 8640 insertions(+), 3856 deletions(-)
----------------------------------------------------------------------



[17/50] [abbrv] ignite git commit: IGNITE-4470 Added support for log file configuration via environmental variable IGNITE_ODBC_LOG_PATH

Posted by sb...@apache.org.
IGNITE-4470 Added support for log file configuration via environmental variable IGNITE_ODBC_LOG_PATH

This closes #1384


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 825fe9cd64ded64068c573d7e3c3901efae6283f
Parents: 9e8e979
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Tue Jan 10 15:57:05 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 15:57:05 2017 +0300

----------------------------------------------------------------------
 modules/platforms/cpp/odbc/Makefile.am          |   3 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++++++++++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/utility.h      |  19 +--
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 .../cpp/odbc/src/config/connection_info.cpp     |   3 -
 modules/platforms/cpp/odbc/src/connection.cpp   |  19 ++-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 +++---
 modules/platforms/cpp/odbc/src/log.cpp          |  83 +++++++++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 167 ++++++++++---------
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 ++-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  11 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |  18 +-
 24 files changed, 412 insertions(+), 209 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/Makefile.am b/modules/platforms/cpp/odbc/Makefile.am
index 1eb42d1..de6f75a 100644
--- a/modules/platforms/cpp/odbc/Makefile.am
+++ b/modules/platforms/cpp/odbc/Makefile.am
@@ -79,7 +79,8 @@ libignite_odbc_la_SOURCES = \
     src/column.cpp \
     src/statement.cpp \
     src/type_traits.cpp \
-    src/utility.cpp
+    src/utility.cpp \
+    src/log.cpp
 
 clean-local:
 	$(RM) *.gcno *.gcda

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/Makefile.am
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/Makefile.am b/modules/platforms/cpp/odbc/include/Makefile.am
index 073dcaa..36d8e25 100644
--- a/modules/platforms/cpp/odbc/include/Makefile.am
+++ b/modules/platforms/cpp/odbc/include/Makefile.am
@@ -53,7 +53,8 @@ noinst_HEADERS = \
     ignite/odbc/cursor.h \
     ignite/odbc/common_types.h \
     ignite/odbc/result_page.h \
-    ignite/odbc/type_traits.h
+    ignite/odbc/type_traits.h \
+    ignite/odbc/log.h
 
 uninstall-hook:
 	if [ -d ${includedir}/ignite ]; then find ${includedir}/ignite -type d -empty -delete; fi

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
index e0ed152..d421952 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/config/connection_info.h
@@ -40,7 +40,6 @@ namespace ignite
                 /** Info type. */
                 typedef unsigned short InfoType;
 
-#ifdef ODBC_DEBUG
                 /**
                  * Convert type to string containing its name.
                  * Debug function.
@@ -48,7 +47,6 @@ namespace ignite
                  * @return Null-terminated string containing types name.
                  */
                 static const char* InfoTypeToString(InfoType type);
-#endif
 
                 /**
                  * Constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/log.h b/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
new file mode 100644
index 0000000..5a6dafc
--- /dev/null
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/log.h
@@ -0,0 +1,123 @@
+/*
+ * 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.
+ */
+
+#ifndef _IGNITE_ODBC_LOG
+#define _IGNITE_ODBC_LOG
+
+#include <string>
+#include <sstream>
+#include <fstream>
+
+#include "ignite/common/common.h"
+#include "ignite/common/concurrent.h"
+
+#   define LOG_MSG(param)                                      \
+    if (ignite::odbc::Logger* p = ignite::odbc::Logger::Get()) \
+    {                                                          \
+        ignite::odbc::LogStream lstream(p);                    \
+        lstream << __FUNCTION__ << ": " << param;              \
+    }
+
+namespace ignite
+{
+    namespace odbc
+    {
+        /* Forward declaration */
+        class Logger;
+
+        /**
+         * Helper object providing stream operations for single log line.
+         * Writes resulting string to Logger object upon destruction.
+         */
+        class LogStream: public std::basic_ostream<char>
+        {
+        public:
+            /**
+             * Constructor.
+             * @param parent pointer to Logger.
+             */
+            LogStream(Logger* parent);
+
+            /**
+             * Conversion operator helpful to determine if log is enabled
+             * @return True if logger is enabled
+             */
+            bool operator()();
+
+            /**
+             * Destructor.
+             */
+            virtual ~LogStream();
+
+        private:
+            IGNITE_NO_COPY_ASSIGNMENT(LogStream);
+
+            /** String buffer. */
+            std::basic_stringbuf<char> strbuf;
+
+            /** Parent logger object */
+            Logger* logger;
+        };
+
+        /**
+         * Logging facility.
+         */
+        class Logger
+        {
+        public:
+            /**
+             * Get instance of Logger, if enabled.
+             * @return Logger instance if logging is enabled. Null otherwise.
+             */
+            static Logger* Get();
+
+            /**
+             * Checks if logging is enabled.
+             * @return True, if logging is enabled.
+             */
+            bool IsEnabled() const;
+
+            /**
+             * Outputs the message to log file
+             * @param message The message to write
+             */
+            void WriteMessage(std::string const& message);
+
+        private:
+            /**
+             * Constructor.
+             * @param path to log file.
+             */
+            Logger(const char* path);
+
+            /**
+             * Destructor.
+             */
+            ~Logger();
+
+            IGNITE_NO_COPY_ASSIGNMENT(Logger);
+
+            /** Mutex for writes synchronization. */
+            ignite::common::concurrent::CriticalSection mutex;
+
+            /** File stream. */
+            std::ofstream stream;
+        };
+    }
+}
+
+#endif //_IGNITE_ODBC_LOG

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h b/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
index 5ffff03..6a94b04 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/meta/column_meta.h
@@ -38,7 +38,6 @@ namespace ignite
             class ColumnMeta
             {
             public:
-#ifdef ODBC_DEBUG
                 /**
                  * Convert attribute ID to string containing its name.
                  * Debug function.
@@ -46,7 +45,6 @@ namespace ignite
                  * @return Null-terminated string containing attribute name.
                  */
                 static const char* AttrIdToString(uint16_t id);
-#endif
 
                 /**
                  * Default constructor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
index 590e97c..7ff5b23 100644
--- a/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
+++ b/modules/platforms/cpp/odbc/include/ignite/odbc/utility.h
@@ -27,6 +27,7 @@
 
 #include <string>
 #include <algorithm>
+#include <sstream>
 
 #include <ignite/common/utils.h>
 #include <ignite/common/decimal.h>
@@ -34,22 +35,6 @@
 #include "ignite/impl/binary/binary_reader_impl.h"
 #include "ignite/impl/binary/binary_writer_impl.h"
 
-
-#ifdef ODBC_DEBUG
-
-extern FILE* log_file;
-void logInit(const char*);
-
-#   define LOG_MSG(fmt, ...)                                        \
-    do {                                                            \
-        logInit(ODBC_LOG_PATH);                                     \
-        fprintf(log_file, "%s: " fmt, __FUNCTION__, ##__VA_ARGS__);   \
-        fflush(log_file);                                           \
-    } while (false)
-#else
-#   define LOG_MSG(...)
-#endif
-
 namespace ignite
 {
     namespace utility
@@ -187,7 +172,7 @@ namespace ignite
          * @param count data length
          * @return standard string containing the formated hex dump
          */
-        std::string HexDump(const char* data, size_t count);
+        std::string HexDump(const void* data, size_t count);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp b/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
index a355625..9bdf1d7 100644
--- a/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
+++ b/modules/platforms/cpp/odbc/os/linux/src/system/socket_client.cpp
@@ -27,6 +27,7 @@
 
 #include "ignite/odbc/system/socket_client.h"
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 
 #define SOCKET_ERROR (-1)
 
@@ -49,7 +50,7 @@ namespace ignite
 
             bool SocketClient::Connect(const char* hostname, uint16_t port)
             {
-                LOG_MSG("Host: %s, port: %d\n", hostname, port);
+                LOG_MSG("Host: " << hostname << ", port: " << port);
 
                 addrinfo hints;
                 memset(&hints, 0, sizeof(hints));
@@ -70,8 +71,10 @@ namespace ignite
                 // Attempt to connect to an address until one succeeds
                 for (addrinfo *it = result; it != NULL; it = it->ai_next) 
                 {
-                    LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2], it->ai_addr->sa_data[3],
-                                                   it->ai_addr->sa_data[4], it->ai_addr->sa_data[5]);
+                    LOG_MSG("Addr: " << it->ai_addr->sa_data[2] << "."
+                                     << it->ai_addr->sa_data[3] << "."
+                                     << it->ai_addr->sa_data[4] << "."
+                                     << it->ai_addr->sa_data[5]);
 
                     // Create a SOCKET for connecting to server
                     socketHandle = socket(it->ai_family, it->ai_socktype, it->ai_protocol);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
index e248323..4c440e2 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system/socket_client.cpp
@@ -28,6 +28,7 @@
 
 #include "ignite/odbc/system/socket_client.h"
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 
 namespace ignite
 {
@@ -64,7 +65,7 @@ namespace ignite
                 addrinfo *result = NULL;
                 addrinfo hints;
 
-                LOG_MSG("Host: %s, port: %d\n", hostname, port);
+                LOG_MSG("Host: " << hostname << " port: " << port);
 
                 memset(&hints, 0, sizeof(hints));
                 hints.ai_family = AF_UNSPEC;
@@ -83,8 +84,10 @@ namespace ignite
                 // Attempt to connect to an address until one succeeds
                 for (addrinfo *it = result; it != NULL; it = it->ai_next)
                 {
-                    LOG_MSG("Addr: %u.%u.%u.%u\n", it->ai_addr->sa_data[2] & 0xFF, it->ai_addr->sa_data[3] & 0xFF,
-                                                   it->ai_addr->sa_data[4] & 0xFF, it->ai_addr->sa_data[5] & 0xFF);
+                    LOG_MSG("Addr: " << (it->ai_addr->sa_data[2] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[3] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[4] & 0xFF) << "."
+                                     << (it->ai_addr->sa_data[5] & 0xFF));
 
                     // Create a SOCKET for connecting to server
                     socketHandle = socket(it->ai_family, it->ai_socktype, it->ai_protocol);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
index a758bd9..864ca85 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system/ui/dsn_configuration_window.cpp
@@ -17,6 +17,8 @@
 
 #include <Windowsx.h>
 
+#include "ignite/odbc/log.h"
+
 #include "ignite/odbc/system/ui/dsn_configuration_window.h"
 
 namespace ignite
@@ -300,14 +302,14 @@ namespace ignite
                     distributedJoins = distributedJoinsCheckBox->IsEnabled() && distributedJoinsCheckBox->IsChecked();
                     enforceJoinOrder = enforceJoinOrderCheckBox->IsEnabled() && enforceJoinOrderCheckBox->IsChecked();
 
-                    LOG_MSG("Retriving arguments:\n");
-                    LOG_MSG("DSN:                %s\n", dsn.c_str());
-                    LOG_MSG("Address:            %s\n", address.c_str());
-                    LOG_MSG("Cache:              %s\n", cache.c_str());
-                    LOG_MSG("Page size:          %d\n", pageSize);
-                    LOG_MSG("Protocol version:   %s\n", version.c_str());
-                    LOG_MSG("Distributed Joins:  %s\n", distributedJoins ? "true" : "false");
-                    LOG_MSG("Enforce Join Order: %s\n", enforceJoinOrder ? "true" : "false");
+                    LOG_MSG("Retriving arguments:");
+                    LOG_MSG("DSN:                " << dsn);
+                    LOG_MSG("Address:            " << address);
+                    LOG_MSG("Cache:              " << cache);
+                    LOG_MSG("Page size:          " << pageSize);
+                    LOG_MSG("Protocol version:   " << version);
+                    LOG_MSG("Distributed Joins:  " << (distributedJoins ? "true" : "false"));
+                    LOG_MSG("Enforce Join Order: " << (enforceJoinOrder ? "true" : "false"));
 
                     if (dsn.empty())
                         throw IgniteError(IgniteError::IGNITE_ERR_GENERIC, "DSN name can not be empty.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
index f432a40..3d41e3e 100644
--- a/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
+++ b/modules/platforms/cpp/odbc/os/win/src/system_dsn.cpp
@@ -16,6 +16,7 @@
  */
 
 #include "ignite/odbc/utility.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
 #include "ignite/odbc/dsn_config.h"
@@ -147,27 +148,27 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 {
     using namespace ignite::odbc;
 
-    LOG_MSG("ConfigDSN called\n");
+    LOG_MSG("ConfigDSN called");
 
     Configuration config;
 
-    LOG_MSG("Attributes: %s\n", attributes);
+    LOG_MSG("Attributes: " << attributes);
 
     config.FillFromConfigAttributes(attributes);
 
     if (!SQLValidDSN(config.GetDsn().c_str()))
         return FALSE;
 
-    LOG_MSG("Driver: %s\n", driver);
-    LOG_MSG("Attributes: %s\n", attributes);
+    LOG_MSG("Driver: " << driver);
+    LOG_MSG("Attributes: " << attributes);
 
-    LOG_MSG("DSN: %s\n", config.GetDsn().c_str());
+    LOG_MSG("DSN: " << config.GetDsn());
 
     switch (req)
     {
         case ODBC_ADD_DSN:
         {
-            LOG_MSG("ODBC_ADD_DSN\n");
+            LOG_MSG("ODBC_ADD_DSN");
 
             if (!DisplayConfigureDsnWindow(hwndParent, config))
                 return FALSE;
@@ -180,7 +181,7 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 
         case ODBC_CONFIG_DSN:
         {
-            LOG_MSG("ODBC_CONFIG_DSN\n");
+            LOG_MSG("ODBC_CONFIG_DSN");
 
             std::string dsn = config.GetDsn();
 
@@ -202,7 +203,7 @@ BOOL INSTAPI ConfigDSN(HWND hwndParent, WORD req, LPCSTR driver, LPCSTR attribut
 
         case ODBC_REMOVE_DSN:
         {
-            LOG_MSG("ODBC_REMOVE_DSN\n");
+            LOG_MSG("ODBC_REMOVE_DSN");
 
             if (!UnregisterDsn(config.GetDsn().c_str()))
                 return FALSE;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
index 56358c5..b00e432 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj
@@ -188,6 +188,7 @@
     <ClCompile Include="..\..\src\statement.cpp" />
     <ClCompile Include="..\..\src\type_traits.cpp" />
     <ClCompile Include="..\..\src\utility.cpp" />
+    <ClCompile Include="..\..\src\log.cpp" />
   </ItemGroup>
   <ItemGroup>
     <None Include="module.def" />
@@ -230,6 +231,7 @@
     <ClInclude Include="..\..\include\ignite\odbc\system\ui\dsn_configuration_window.h" />
     <ClInclude Include="..\..\include\ignite\odbc\type_traits.h" />
     <ClInclude Include="..\..\include\ignite\odbc\utility.h" />
+    <ClInclude Include="..\..\include\ignite\odbc\log.h" />
     <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\custom_window.h" />
     <ClInclude Include="..\..\os\win\include\ignite\odbc\system\ui\window.h" />
   </ItemGroup>

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
index 58764e4..1828be1 100644
--- a/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
+++ b/modules/platforms/cpp/odbc/project/vs/odbc.vcxproj.filters
@@ -52,6 +52,9 @@
     <ClCompile Include="..\..\src\utility.cpp">
       <Filter>Code</Filter>
     </ClCompile>
+    <ClCompile Include="..\..\src\log.cpp">
+      <Filter>Code</Filter>
+    </ClCompile>
     <ClCompile Include="..\..\src\cursor.cpp">
       <Filter>Code</Filter>
     </ClCompile>
@@ -176,6 +179,9 @@
     <ClInclude Include="..\..\include\ignite\odbc\utility.h">
       <Filter>Code</Filter>
     </ClInclude>
+    <ClInclude Include="..\..\include\ignite\odbc\log.h">
+      <Filter>Code</Filter>
+    </ClInclude>
     <ClInclude Include="..\..\include\ignite\odbc\query\query.h">
       <Filter>Code\query</Filter>
     </ClInclude>

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/config/connection_info.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/config/connection_info.cpp b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
index 7c612bc..ed76aab 100644
--- a/modules/platforms/cpp/odbc/src/config/connection_info.cpp
+++ b/modules/platforms/cpp/odbc/src/config/connection_info.cpp
@@ -42,8 +42,6 @@ namespace ignite
         namespace config
         {
 
-#ifdef ODBC_DEBUG
-
 #define DBG_STR_CASE(x) case x: return #x
 
             const char * ConnectionInfo::InfoTypeToString(InfoType type)
@@ -102,7 +100,6 @@ namespace ignite
             }
 
 #undef DBG_STR_CASE
-#endif
 
             ConnectionInfo::ConnectionInfo() : strParams(), intParams(),
                 shortParams()

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index 4a6de5e..960a507 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -21,6 +21,7 @@
 
 #include <ignite/common/fixed_size_array.h>
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/statement.h"
 #include "ignite/odbc/connection.h"
@@ -65,6 +66,13 @@ namespace ignite
 
         void Connection::GetInfo(config::ConnectionInfo::InfoType type, void* buf, short buflen, short* reslen)
         {
+            LOG_MSG("SQLGetInfo called: "
+                << type << " ("
+                << config::ConnectionInfo::InfoTypeToString(type) << "), "
+                << std::hex << reinterpret_cast<size_t>(buf) << ", "
+                << buflen << ", "
+                << std::hex << reinterpret_cast<size_t>(reslen));
+
             IGNITE_ODBC_API_CALL(InternalGetInfo(type, buf, buflen, reslen));
         }
 
@@ -195,7 +203,7 @@ namespace ignite
             if (sent != len + sizeof(OdbcProtocolHeader))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message");
 
-            LOG_MSG("message sent: (%d bytes)%s\n", msg.GetSize(), utility::HexDump((char*)msg.GetData(), msg.GetSize()).c_str());
+            LOG_MSG("message sent: (" <<  msg.GetSize() << " bytes)" << utility::HexDump(msg.GetData(), msg.GetSize()));
         }
 
         size_t Connection::SendAll(const int8_t* data, size_t len)
@@ -206,7 +214,7 @@ namespace ignite
             {
                 int res = socket.Send(data + sent, len - sent);
 
-                LOG_MSG("Sent: %d\n", res);
+                LOG_MSG("Sent: " << res);
 
                 if (res <= 0)
                     return sent;
@@ -259,8 +267,7 @@ namespace ignite
                 size_t received = len - remain;
 
                 int res = socket.Receive(buffer + received, remain);
-                LOG_MSG("Receive res: %d\n", res);
-                LOG_MSG("remain: %d\n", remain);
+                LOG_MSG("Receive res: " << res << " remain: " << remain);
 
                 if (res <= 0)
                     return received;
@@ -345,7 +352,7 @@ namespace ignite
 
             if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
             {
-                LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                LOG_MSG("Error: " << rsp.GetError().c_str());
 
                 AddStatusRecord(SQL_STATE_08001_CANNOT_CONNECT, rsp.GetError());
 
@@ -356,7 +363,7 @@ namespace ignite
 
             if (!rsp.IsAccepted())
             {
-                LOG_MSG("Hanshake message has been rejected.\n");
+                LOG_MSG("Hanshake message has been rejected.");
 
                 std::stringstream constructor;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
index 260e3e4..94ea53e 100644
--- a/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
+++ b/modules/platforms/cpp/odbc/src/diagnostic/diagnosable_adapter.cpp
@@ -15,6 +15,7 @@
  * limitations under the License.
  */
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/diagnostic/diagnosable_adapter.h"
 
@@ -41,7 +42,7 @@ namespace ignite
 
             void DiagnosableAdapter::AddStatusRecord(SqlState sqlState, const std::string& message)
             {
-                LOG_MSG("Adding new record: %s\n", message.c_str());
+                LOG_MSG("Adding new record: " << message);
 
                 AddStatusRecord(sqlState, message, 0, 0);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/entry_points.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/entry_points.cpp b/modules/platforms/cpp/odbc/src/entry_points.cpp
index 850fa10..e241a9f 100644
--- a/modules/platforms/cpp/odbc/src/entry_points.cpp
+++ b/modules/platforms/cpp/odbc/src/entry_points.cpp
@@ -17,6 +17,8 @@
 
 #include "ignite/odbc.h"
 
+#include "ignite/odbc/log.h"
+
 #include "ignite/odbc/utility.h"
 
 SQLRETURN SQL_API SQLGetInfo(SQLHDBC        conn,
@@ -440,7 +442,7 @@ SQLRETURN SQL_API SQLError(SQLHENV      env,
 
 SQLRETURN SQL_API SQLCancel(SQLHSTMT stmt)
 {
-    LOG_MSG("SQLCancel called\n");
+    LOG_MSG("SQLCancel called");
     return SQL_SUCCESS;
 }
 
@@ -452,7 +454,7 @@ SQLRETURN SQL_API SQLColAttributes(SQLHSTMT     stmt,
                                    SQLSMALLINT* strAttrResLen,
                                    SQLLEN*      numAttrBuf)
 {
-    LOG_MSG("SQLColAttributes called\n");
+    LOG_MSG("SQLColAttributes called");
     return SQL_SUCCESS;
 }
 
@@ -461,7 +463,7 @@ SQLRETURN SQL_API SQLGetCursorName(SQLHSTMT     stmt,
                                    SQLSMALLINT  nameBufLen,
                                    SQLSMALLINT* nameResLen)
 {
-    LOG_MSG("SQLGetCursorName called\n");
+    LOG_MSG("SQLGetCursorName called");
     return SQL_SUCCESS;
 }
 
@@ -469,7 +471,7 @@ SQLRETURN SQL_API SQLSetCursorName(SQLHSTMT     stmt,
                                    SQLCHAR*     name,
                                    SQLSMALLINT  nameLen)
 {
-    LOG_MSG("SQLSetCursorName called\n");
+    LOG_MSG("SQLSetCursorName called");
     return SQL_SUCCESS;
 }
 
@@ -477,7 +479,7 @@ SQLRETURN SQL_API SQLGetConnectOption(SQLHDBC       conn,
                                       SQLUSMALLINT  option,
                                       SQLPOINTER    value)
 {
-    LOG_MSG("SQLGetConnectOption called\n");
+    LOG_MSG("SQLGetConnectOption called");
     return SQL_SUCCESS;
 }
 
@@ -485,7 +487,7 @@ SQLRETURN SQL_API SQLGetStmtOption(SQLHSTMT     stmt,
                                    SQLUSMALLINT option,
                                    SQLPOINTER   value)
 {
-    LOG_MSG("SQLGetStmtOption called\n");
+    LOG_MSG("SQLGetStmtOption called");
     return SQL_SUCCESS;
 }
 
@@ -493,7 +495,7 @@ SQLRETURN SQL_API SQLSetConnectOption(SQLHDBC       conn,
                                       SQLUSMALLINT  option,
                                       SQLULEN       value)
 {
-    LOG_MSG("SQLSetConnectOption called\n");
+    LOG_MSG("SQLSetConnectOption called");
     return SQL_SUCCESS;
 }
 
@@ -501,7 +503,7 @@ SQLRETURN SQL_API SQLSetStmtOption(SQLHSTMT     stmt,
                                    SQLUSMALLINT option,
                                    SQLULEN      value)
 {
-    LOG_MSG("SQLSetStmtOption called\n");
+    LOG_MSG("SQLSetStmtOption called");
     return SQL_SUCCESS;
 }
 
@@ -515,7 +517,7 @@ SQLRETURN SQL_API SQLStatistics(SQLHSTMT        stmt,
                                 SQLUSMALLINT    unique,
                                 SQLUSMALLINT    reserved)
 {
-    LOG_MSG("SQLStatistics called\n");
+    LOG_MSG("SQLStatistics called");
     return SQL_SUCCESS;
 }
 
@@ -526,7 +528,7 @@ SQLRETURN SQL_API SQLBrowseConnect(SQLHDBC      conn,
                                    SQLSMALLINT  outConnectionStrBufLen,
                                    SQLSMALLINT* outConnectionStrResLen)
 {
-    LOG_MSG("SQLBrowseConnect called\n");
+    LOG_MSG("SQLBrowseConnect called");
     return SQL_SUCCESS;
 }
 
@@ -540,7 +542,7 @@ SQLRETURN SQL_API SQLProcedureColumns(SQLHSTMT      stmt,
                                       SQLCHAR *     columnName,
                                       SQLSMALLINT   columnNameLen)
 {
-    LOG_MSG("SQLProcedureColumns called\n");
+    LOG_MSG("SQLProcedureColumns called");
     return SQL_SUCCESS;
 }
 
@@ -549,7 +551,7 @@ SQLRETURN SQL_API SQLSetPos(SQLHSTMT        stmt,
                             SQLUSMALLINT    operation,
                             SQLUSMALLINT    lockType)
 {
-    LOG_MSG("SQLSetPos called\n");
+    LOG_MSG("SQLSetPos called");
     return SQL_SUCCESS;
 }
 
@@ -558,7 +560,7 @@ SQLRETURN SQL_API SQLSetScrollOptions(SQLHSTMT      stmt,
                                       SQLLEN        crowKeyset,
                                       SQLUSMALLINT  crowRowset)
 {
-    LOG_MSG("SQLSetScrollOptions called\n");
+    LOG_MSG("SQLSetScrollOptions called");
     return SQL_SUCCESS;
 }
 
@@ -568,7 +570,7 @@ SQLRETURN SQL_API SQLGetConnectAttr(SQLHDBC     conn,
                                     SQLINTEGER  valueBufLen,
                                     SQLINTEGER* valueResLen)
 {
-    LOG_MSG("SQLGetConnectAttr called\n");
+    LOG_MSG("SQLGetConnectAttr called");
     return SQL_SUCCESS;
 }
 
@@ -577,14 +579,14 @@ SQLRETURN SQL_API SQLSetConnectAttr(SQLHDBC     conn,
                                     SQLPOINTER  value,
                                     SQLINTEGER  valueLen)
 {
-    LOG_MSG("SQLSetConnectAttr called\n");
+    LOG_MSG("SQLSetConnectAttr called");
     return SQL_SUCCESS;
 }
 
 SQLRETURN SQL_API SQLBulkOperations(SQLHSTMT       stmt,
                                     SQLUSMALLINT   operation)
 {
-    LOG_MSG("SQLBulkOperations called\n");
+    LOG_MSG("SQLBulkOperations called");
     return SQL_SUCCESS;
 }
 
@@ -596,13 +598,13 @@ SQLRETURN SQL_API SQLTablePrivileges(SQLHSTMT      stmt,
                                      SQLCHAR*      tableName,
                                      SQLSMALLINT   tableNameLen)
 {
-    LOG_MSG("SQLTablePrivileges called\n");
+    LOG_MSG("SQLTablePrivileges called");
     return SQL_SUCCESS;
 }
 
 SQLRETURN SQL_API SQLCopyDesc(SQLHDESC src, SQLHDESC dst)
 {
-    LOG_MSG("SQLCopyDesc called\n");
+    LOG_MSG("SQLCopyDesc called");
     return SQL_SUCCESS;
 }
 
@@ -613,7 +615,7 @@ SQLRETURN SQL_API SQLGetDescField(SQLHDESC      descr,
                                   SQLINTEGER    bufferLen,
                                   SQLINTEGER*   resLen)
 {
-    LOG_MSG("SQLGetDescField called\n");
+    LOG_MSG("SQLGetDescField called");
     return SQL_SUCCESS;
 }
 
@@ -629,7 +631,7 @@ SQLRETURN SQL_API SQLGetDescRec(SQLHDESC        DescriptorHandle,
                                 SQLSMALLINT*    scale,
                                 SQLSMALLINT*    nullable)
 {
-    LOG_MSG("SQLGetDescRec called\n");
+    LOG_MSG("SQLGetDescRec called");
     return SQL_SUCCESS;
 }
 
@@ -639,7 +641,7 @@ SQLRETURN SQL_API SQLSetDescField(SQLHDESC      descr,
                                   SQLPOINTER    buffer,
                                   SQLINTEGER    bufferLen)
 {
-    LOG_MSG("SQLSetDescField called\n");
+    LOG_MSG("SQLSetDescField called");
     return SQL_SUCCESS;
 }
 
@@ -654,7 +656,7 @@ SQLRETURN SQL_API SQLSetDescRec(SQLHDESC      descr,
                                 SQLLEN*       resLen,
                                 SQLLEN*       id)
 {
-    LOG_MSG("SQLSetDescRec called\n");
+    LOG_MSG("SQLSetDescRec called");
     return SQL_SUCCESS;
 }
 
@@ -668,7 +670,7 @@ SQLRETURN SQL_API SQLColumnPrivileges(SQLHSTMT      stmt,
                                       SQLCHAR*      columnName,
                                       SQLSMALLINT   columnNameLen)
 {
-    LOG_MSG("SQLColumnPrivileges called\n");
+    LOG_MSG("SQLColumnPrivileges called");
     return SQL_SUCCESS;
 }
 
@@ -676,7 +678,7 @@ SQLRETURN SQL_API SQLParamOptions(SQLHSTMT  stmt,
                                   SQLULEN   paramSetSize,
                                   SQLULEN*  paramsProcessed)
 {
-    LOG_MSG("SQLParamOptions called\n");
+    LOG_MSG("SQLParamOptions called");
     return SQL_SUCCESS;
 }
 
@@ -688,6 +690,6 @@ SQLRETURN SQL_API SQLProcedures(SQLHSTMT        stmt,
                                 SQLCHAR*        tableName,
                                 SQLSMALLINT     tableNameLen)
 {
-    LOG_MSG("SQLProcedures called\n");
+    LOG_MSG("SQLProcedures called");
     return SQL_SUCCESS;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/log.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/log.cpp b/modules/platforms/cpp/odbc/src/log.cpp
new file mode 100644
index 0000000..755798f
--- /dev/null
+++ b/modules/platforms/cpp/odbc/src/log.cpp
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+
+#include <cstdlib>
+
+#include "ignite/odbc/log.h"
+
+namespace ignite
+{
+    namespace odbc
+    {
+        LogStream::LogStream(Logger* parent) :
+            std::basic_ostream<char>(0),
+            strbuf(),
+            logger(parent)
+        {
+            init(&strbuf);
+        }
+
+        bool LogStream::operator()()
+        {
+            return logger != 0;
+        }
+
+        LogStream::~LogStream()
+        {
+            if (logger)
+            {
+                logger->WriteMessage(strbuf.str());
+            }
+        }
+
+        Logger::Logger(const char* path) :
+            mutex(),
+            stream()
+        {
+            if (path)
+            {
+                stream.open(path);
+            }
+        }
+
+        Logger::~Logger()
+        {
+        }
+
+        bool Logger::IsEnabled() const
+        {
+            return stream.is_open();
+        }
+
+        void Logger::WriteMessage(std::string const& message)
+        {
+            if (IsEnabled())
+            {
+                ignite::common::concurrent::CsLockGuard guard(mutex);
+                stream << message << std::endl;
+            }
+        }
+
+        Logger* Logger::Get()
+        {
+            const char* envVarName = "IGNITE_ODBC_LOG_PATH";
+            static Logger logger(getenv(envVarName));
+            return logger.IsEnabled() ? &logger : 0;
+        }
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/meta/column_meta.cpp b/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
index d397618..12dbfc1 100644
--- a/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
+++ b/modules/platforms/cpp/odbc/src/meta/column_meta.cpp
@@ -27,8 +27,6 @@ namespace ignite
         namespace meta
         {
 
-#ifdef ODBC_DEBUG
-
 #define DBG_STR_CASE(x) case x: return #x
 
             const char* ColumnMeta::AttrIdToString(uint16_t id)
@@ -70,8 +68,6 @@ namespace ignite
 
 #undef DBG_STR_CASE
 
-#endif
-
             void ColumnMeta::Read(ignite::impl::binary::BinaryReaderImpl& reader)
             {
                 utility::ReadString(reader, schemaName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/odbc.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/odbc.cpp b/modules/platforms/cpp/odbc/src/odbc.cpp
index fd69c0d..542e64c 100644
--- a/modules/platforms/cpp/odbc/src/odbc.cpp
+++ b/modules/platforms/cpp/odbc/src/odbc.cpp
@@ -20,6 +20,7 @@
 #include <cstring>
 #include <algorithm>
 
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
@@ -43,9 +44,10 @@ namespace ignite
         using odbc::Connection;
         using odbc::config::ConnectionInfo;
 
-        LOG_MSG("SQLGetInfo called: %d (%s), %p, %d, %p\n",
-                infoType, ConnectionInfo::InfoTypeToString(infoType),
-                infoValue, infoValueMax, length);
+        LOG_MSG("SQLGetInfo called: "
+            << infoType << " (" << ConnectionInfo::InfoTypeToString(infoType) << "), "
+            << std::hex << reinterpret_cast<size_t>(infoValue) << ", " << infoValueMax << ", "
+            << std::hex << reinterpret_cast<size_t>(length));
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -59,7 +61,7 @@ namespace ignite
 
     SQLRETURN SQLAllocHandle(SQLSMALLINT type, SQLHANDLE parent, SQLHANDLE* result)
     {
-        //LOG_MSG("SQLAllocHandle called\n");
+        //LOG_MSG("SQLAllocHandle called");
         switch (type)
         {
             case SQL_HANDLE_ENV:
@@ -101,7 +103,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLAllocEnv called\n");
+        LOG_MSG("SQLAllocEnv called");
 
         *env = reinterpret_cast<SQLHENV>(new Environment());
 
@@ -113,7 +115,7 @@ namespace ignite
         using odbc::Environment;
         using odbc::Connection;
 
-        LOG_MSG("SQLAllocConnect called\n");
+        LOG_MSG("SQLAllocConnect called");
 
         *conn = SQL_NULL_HDBC;
 
@@ -137,7 +139,7 @@ namespace ignite
         using odbc::Connection;
         using odbc::Statement;
 
-        LOG_MSG("SQLAllocStmt called\n");
+        LOG_MSG("SQLAllocStmt called");
 
         *stmt = SQL_NULL_HDBC;
 
@@ -178,7 +180,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLFreeEnv called\n");
+        LOG_MSG("SQLFreeEnv called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -194,7 +196,7 @@ namespace ignite
     {
         using odbc::Connection;
 
-        LOG_MSG("SQLFreeConnect called\n");
+        LOG_MSG("SQLFreeConnect called");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -210,7 +212,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFreeStmt called\n");
+        LOG_MSG("SQLFreeStmt called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -232,7 +234,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLCloseCursor called\n");
+        LOG_MSG("SQLCloseCursor called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -257,8 +259,9 @@ namespace ignite
 
         UNREFERENCED_PARAMETER(windowHandle);
 
-        LOG_MSG("SQLDriverConnect called\n");
-        LOG_MSG("Connection String: [%s]\n", inConnectionString);
+        LOG_MSG("SQLDriverConnect called");
+        if (inConnectionString)
+            LOG_MSG("Connection String: [" << inConnectionString << "]");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -290,7 +293,8 @@ namespace ignite
         if (outConnectionStringLen)
             *outConnectionStringLen = static_cast<SQLSMALLINT>(reslen);
 
-        LOG_MSG("%s\n", outConnectionString);
+        if (outConnectionString)
+            LOG_MSG(outConnectionString);
 
         return diag.GetReturnCode();
     }
@@ -318,7 +322,7 @@ namespace ignite
 
         std::string dsn = SqlStringToString(serverName, serverNameLen);
 
-        LOG_MSG("DSN: %s\n", dsn.c_str());
+        LOG_MSG("DSN: " << dsn);
 
         odbc::ReadDsnConfiguration(dsn.c_str(), config);
 
@@ -331,7 +335,7 @@ namespace ignite
     {
         using odbc::Connection;
 
-        LOG_MSG("SQLDisconnect called\n");
+        LOG_MSG("SQLDisconnect called");
 
         Connection *connection = reinterpret_cast<Connection*>(conn);
 
@@ -348,7 +352,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLPrepare called\n");
+        LOG_MSG("SQLPrepare called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -357,7 +361,7 @@ namespace ignite
 
         std::string sql = SqlStringToString(query, queryLen);
 
-        LOG_MSG("SQL: %s\n", sql.c_str());
+        LOG_MSG("SQL: " << sql);
 
         statement->PrepareSqlQuery(sql);
 
@@ -368,7 +372,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLExecute called\n");
+        LOG_MSG("SQLExecute called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -385,7 +389,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLExecDirect called\n");
+        LOG_MSG("SQLExecDirect called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -394,7 +398,7 @@ namespace ignite
 
         std::string sql = SqlStringToString(query, queryLen);
 
-        LOG_MSG("SQL: %s\n", sql.c_str());
+        LOG_MSG("SQL: " << sql);
 
         statement->ExecuteSqlQuery(sql);
 
@@ -413,7 +417,10 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLBindCol called: index=%d, type=%d targetValue=%p bufferLength=%d\n", colNum, targetType, targetValue, bufferLength);
+        LOG_MSG("SQLBindCol called: index=" << colNum << ", type=" << targetType << 
+                ", targetValue=" << reinterpret_cast<size_t>(targetValue) << 
+                ", bufferLength=" << bufferLength << 
+                ", lengthInd=" << reinterpret_cast<size_t>(strLengthOrIndicator));
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -429,7 +436,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFetch called\n");
+        LOG_MSG("SQLFetch called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -445,8 +452,8 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLFetchScroll called\n");
-        LOG_MSG("Orientation: %d, Offset: %d\n", orientation, offset);
+        LOG_MSG("SQLFetchScroll called");
+        LOG_MSG("Orientation: " << orientation << " Offset: " << offset);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -464,7 +471,7 @@ namespace ignite
                                SQLULEN*         rowCount,
                                SQLUSMALLINT*    rowStatusArray)
     {
-        LOG_MSG("SQLExtendedFetch called\n");
+        LOG_MSG("SQLExtendedFetch called");
 
         SQLRETURN res = SQLFetchScroll(stmt, orientation, offset);
 
@@ -487,7 +494,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::meta::ColumnMetaVector;
 
-        LOG_MSG("SQLNumResultCols called\n");
+        LOG_MSG("SQLNumResultCols called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -499,7 +506,7 @@ namespace ignite
         if (columnNum)
         {
             *columnNum = static_cast<SQLSMALLINT>(res);
-            LOG_MSG("columnNum: %d\n", *columnNum);
+            LOG_MSG("columnNum: " << *columnNum);
         }
 
         return statement->GetDiagnosticRecords().GetReturnCode();
@@ -518,7 +525,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLTables called\n");
+        LOG_MSG("SQLTables called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -530,10 +537,10 @@ namespace ignite
         std::string table = SqlStringToString(tableName, tableNameLen);
         std::string tableTypeStr = SqlStringToString(tableType, tableTypeLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
-        LOG_MSG("tableType: %s\n", tableTypeStr.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
+        LOG_MSG("tableType: " << tableTypeStr);
 
         statement->ExecuteGetTablesMetaQuery(catalog, schema, table, tableTypeStr);
 
@@ -553,7 +560,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLColumns called\n");
+        LOG_MSG("SQLColumns called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -565,10 +572,10 @@ namespace ignite
         std::string table = SqlStringToString(tableName, tableNameLen);
         std::string column = SqlStringToString(columnName, columnNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
-        LOG_MSG("column: %s\n", column.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
+        LOG_MSG("column: " << column);
 
         statement->ExecuteGetColumnsMetaQuery(schema, table, column);
 
@@ -579,7 +586,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLMoreResults called\n");
+        LOG_MSG("SQLMoreResults called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -604,7 +611,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLBindParameter called: %d, %d, %d\n", paramIdx, bufferType, paramSqlType);
+        LOG_MSG("SQLBindParameter called: " << paramIdx << ", " << bufferType << ", " << paramSqlType);
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -625,7 +632,7 @@ namespace ignite
     {
         using namespace utility;
 
-        LOG_MSG("SQLNativeSql called\n");
+        LOG_MSG("SQLNativeSql called");
 
         std::string in = SqlStringToString(inQuery, inQueryLen);
 
@@ -650,7 +657,7 @@ namespace ignite
         using odbc::meta::ColumnMetaVector;
         using odbc::meta::ColumnMeta;
 
-        LOG_MSG("SQLColAttribute called: %d (%s)\n", fieldId, ColumnMeta::AttrIdToString(fieldId));
+        LOG_MSG("SQLColAttribute called: " << fieldId << " (" << ColumnMeta::AttrIdToString(fieldId) << ")");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -689,7 +696,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::SqlLen;
 
-        LOG_MSG("SQLDescribeCol called\n");
+        LOG_MSG("SQLDescribeCol called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -709,13 +716,13 @@ namespace ignite
         statement->GetColumnAttribute(columnNum, SQL_DESC_SCALE, 0, 0, 0, &decimalDigitsRes);
         statement->GetColumnAttribute(columnNum, SQL_DESC_NULLABLE, 0, 0, 0, &nullableRes);
 
-        LOG_MSG("columnNum: %lld\n", columnNum);
-        LOG_MSG("dataTypeRes: %lld\n", dataTypeRes);
-        LOG_MSG("columnSizeRes: %lld\n", columnSizeRes);
-        LOG_MSG("decimalDigitsRes: %lld\n", decimalDigitsRes);
-        LOG_MSG("nullableRes: %lld\n", nullableRes);
-        LOG_MSG("columnNameBuf: %s\n", columnNameBuf ? reinterpret_cast<const char*>(columnNameBuf) : "<null>");
-        LOG_MSG("columnNameLen: %d\n", columnNameLen ? *columnNameLen : -1);
+        LOG_MSG("columnNum: " << columnNum);
+        LOG_MSG("dataTypeRes: " << dataTypeRes);
+        LOG_MSG("columnSizeRes: " << columnSizeRes);
+        LOG_MSG("decimalDigitsRes: " << decimalDigitsRes);
+        LOG_MSG("nullableRes: " << nullableRes);
+        LOG_MSG("columnNameBuf: " << (columnNameBuf ? reinterpret_cast<const char*>(columnNameBuf) : "<null>"));
+        LOG_MSG("columnNameLen: " << (columnNameLen ? *columnNameLen : -1));
 
         if (dataType)
             *dataType = static_cast<SQLSMALLINT>(dataTypeRes);
@@ -737,7 +744,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLRowCount called\n");
+        LOG_MSG("SQLRowCount called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -769,7 +776,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLForeignKeys called\n");
+        LOG_MSG("SQLForeignKeys called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -783,12 +790,12 @@ namespace ignite
         std::string foreignSchema = SqlStringToString(foreignSchemaName, foreignSchemaNameLen);
         std::string foreignTable = SqlStringToString(foreignTableName, foreignTableNameLen);
 
-        LOG_MSG("primaryCatalog: %s\n", primaryCatalog.c_str());
-        LOG_MSG("primarySchema: %s\n", primarySchema.c_str());
-        LOG_MSG("primaryTable: %s\n", primaryTable.c_str());
-        LOG_MSG("foreignCatalog: %s\n", foreignCatalog.c_str());
-        LOG_MSG("foreignSchema: %s\n", foreignSchema.c_str());
-        LOG_MSG("foreignTable: %s\n", foreignTable.c_str());
+        LOG_MSG("primaryCatalog: " << primaryCatalog);
+        LOG_MSG("primarySchema: " << primarySchema);
+        LOG_MSG("primaryTable: " << primaryTable);
+        LOG_MSG("foreignCatalog: " << foreignCatalog);
+        LOG_MSG("foreignSchema: " << foreignSchema);
+        LOG_MSG("foreignTable: " << foreignTable);
 
         statement->ExecuteGetForeignKeysQuery(primaryCatalog, primarySchema,
             primaryTable, foreignCatalog, foreignSchema, foreignTable);
@@ -809,7 +816,7 @@ namespace ignite
 #ifdef ODBC_DEBUG
         using odbc::type_traits::StatementAttrIdToString;
 
-        LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr);
+        LOG_MSG("Attr: " << StatementAttrIdToString(attr) << " (" << attr << ")");
 #endif //ODBC_DEBUG
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
@@ -834,7 +841,7 @@ namespace ignite
 #ifdef ODBC_DEBUG
         using odbc::type_traits::StatementAttrIdToString;
 
-        LOG_MSG("Attr: %s (%d)\n", StatementAttrIdToString(attr), attr);
+        LOG_MSG("Attr: " << StatementAttrIdToString(attr) << " (" << attr << ")");
 #endif //ODBC_DEBUG
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
@@ -858,7 +865,7 @@ namespace ignite
         using odbc::Statement;
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLPrimaryKeys called\n");
+        LOG_MSG("SQLPrimaryKeys called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -869,9 +876,9 @@ namespace ignite
         std::string schema = SqlStringToString(schemaName, schemaNameLen);
         std::string table = SqlStringToString(tableName, tableNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
 
         statement->ExecuteGetPrimaryKeysQuery(catalog, schema, table);
 
@@ -882,7 +889,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLNumParams called\n");
+        LOG_MSG("SQLNumParams called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -909,7 +916,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetDiagField called: %d\n", recNum);
+        LOG_MSG("SQLGetDiagField called: " << recNum);
 
         SqlLen outResLen;
         ApplicationDataBuffer outBuffer(IGNITE_ODBC_C_TYPE_DEFAULT, buffer, bufferLen, &outResLen);
@@ -960,7 +967,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetDiagRec called\n");
+        LOG_MSG("SQLGetDiagRec called");
 
         const DiagnosticRecordStorage* records = 0;
 
@@ -1007,7 +1014,7 @@ namespace ignite
     {
         using odbc::Statement;
 
-        LOG_MSG("SQLGetTypeInfo called\n");
+        LOG_MSG("SQLGetTypeInfo called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1023,7 +1030,7 @@ namespace ignite
     {
         using namespace odbc;
 
-        LOG_MSG("SQLEndTran called\n");
+        LOG_MSG("SQLEndTran called");
 
         SQLRETURN result;
 
@@ -1086,7 +1093,7 @@ namespace ignite
         using odbc::Statement;
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetData called\n");
+        LOG_MSG("SQLGetData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1109,7 +1116,7 @@ namespace ignite
     {
         using odbc::Environment;
 
-        LOG_MSG("SQLSetEnvAttr called\n");
+        LOG_MSG("SQLSetEnvAttr called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -1132,7 +1139,7 @@ namespace ignite
 
         using odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLGetEnvAttr called\n");
+        LOG_MSG("SQLGetEnvAttr called");
 
         Environment *environment = reinterpret_cast<Environment*>(env);
 
@@ -1166,7 +1173,7 @@ namespace ignite
 
         using utility::SqlStringToString;
 
-        LOG_MSG("SQLSpecialColumns called\n");
+        LOG_MSG("SQLSpecialColumns called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1177,9 +1184,9 @@ namespace ignite
         std::string schema = SqlStringToString(schemaName, schemaNameLen);
         std::string table = SqlStringToString(tableName, tableNameLen);
 
-        LOG_MSG("catalog: %s\n", catalog.c_str());
-        LOG_MSG("schema: %s\n", schema.c_str());
-        LOG_MSG("table: %s\n", table.c_str());
+        LOG_MSG("catalog: " << catalog);
+        LOG_MSG("schema: " << schema);
+        LOG_MSG("table: " << table);
 
         statement->ExecuteSpecialColumnsQuery(idType, catalog, schema, table, scope, nullable);
 
@@ -1190,7 +1197,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLParamData called\n");
+        LOG_MSG("SQLParamData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1206,7 +1213,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLPutData called\n");
+        LOG_MSG("SQLPutData called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1227,7 +1234,7 @@ namespace ignite
     {
         using namespace ignite::odbc;
 
-        LOG_MSG("SQLDescribeParam called\n");
+        LOG_MSG("SQLDescribeParam called");
 
         Statement *statement = reinterpret_cast<Statement*>(stmt);
 
@@ -1255,7 +1262,7 @@ namespace ignite
 
         using ignite::odbc::app::ApplicationDataBuffer;
 
-        LOG_MSG("SQLError called\n");
+        LOG_MSG("SQLError called");
 
         SQLHANDLE handle = 0;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
index 1cd2a9f..9aa5f7d 100644
--- a/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/column_metadata_query.cpp
@@ -20,6 +20,7 @@
 #include "ignite/odbc/type_traits.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/column_metadata_query.h"
 
 namespace
@@ -292,8 +293,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
-
+                    LOG_MSG("Error: " << rsp.GetError());
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
                     return SQL_RESULT_ERROR;
@@ -303,11 +303,10 @@ namespace ignite
 
                 for (size_t i = 0; i < meta.size(); ++i)
                 {
-                    LOG_MSG("[%d] SchemaName:     %s\n", i, meta[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TableName:      %s\n", i, meta[i].GetTableName().c_str());
-                    LOG_MSG("[%d] ColumnName:     %s\n", i, meta[i].GetColumnName().c_str());
-                    LOG_MSG("[%d] ColumnType:     %d\n", i, meta[i].GetDataType());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] SchemaName:     " << meta[i].GetSchemaName()
+                         << "\n[" << i << "] TableName:      " << meta[i].GetTableName()
+                         << "\n[" << i << "] ColumnName:     " << meta[i].GetColumnName()
+                         << "\n[" << i << "] ColumnType:     " << meta[i].GetDataType());
                 }
 
                 return SQL_RESULT_SUCCESS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/query/data_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/data_query.cpp b/modules/platforms/cpp/odbc/src/query/data_query.cpp
index 5b1b758..c9762ad 100644
--- a/modules/platforms/cpp/odbc/src/query/data_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/data_query.cpp
@@ -17,6 +17,7 @@
 
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/data_query.h"
 
 namespace ignite
@@ -40,7 +41,7 @@ namespace ignite
             {
                 Close();
             }
-            
+
             SqlResult DataQuery::Execute()
             {
                 if (cursor.get())
@@ -186,7 +187,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -197,15 +198,13 @@ namespace ignite
 
                 resultMeta.assign(rsp.GetMeta().begin(), rsp.GetMeta().end());
 
-                LOG_MSG("Query id: %lld\n", cursor->GetQueryId());
-
+                LOG_MSG("Query id: " << cursor->GetQueryId());
                 for (size_t i = 0; i < rsp.GetMeta().size(); ++i)
                 {
-                    LOG_MSG("[%d] SchemaName:     %s\n", i, rsp.GetMeta()[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TypeName:       %s\n", i, rsp.GetMeta()[i].GetTableName().c_str());
-                    LOG_MSG("[%d] ColumnName:     %s\n", i, rsp.GetMeta()[i].GetColumnName().c_str());
-                    LOG_MSG("[%d] ColumnType:     %d\n", i, rsp.GetMeta()[i].GetDataType());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] SchemaName:     " << rsp.GetMeta()[i].GetSchemaName()
+                        <<  "\n[" << i << "] TypeName:       " << rsp.GetMeta()[i].GetTableName()
+                        <<  "\n[" << i << "] ColumnName:     " << rsp.GetMeta()[i].GetColumnName()
+                        <<  "\n[" << i << "] ColumnType:     " << rsp.GetMeta()[i].GetDataType());
                 }
 
                 return SQL_RESULT_SUCCESS;
@@ -227,11 +226,11 @@ namespace ignite
                     return SQL_RESULT_ERROR;
                 }
 
-                LOG_MSG("Query id: %lld\n", rsp.GetQueryId());
+                LOG_MSG("Query id: " << rsp.GetQueryId());
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -261,7 +260,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
index 3cddd1b..4fd5f7b 100644
--- a/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
+++ b/modules/platforms/cpp/odbc/src/query/table_metadata_query.cpp
@@ -20,6 +20,7 @@
 #include "ignite/odbc/type_traits.h"
 #include "ignite/odbc/connection.h"
 #include "ignite/odbc/message.h"
+#include "ignite/odbc/log.h"
 #include "ignite/odbc/query/table_metadata_query.h"
 
 namespace
@@ -218,7 +219,7 @@ namespace ignite
 
                 if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
                 {
-                    LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                    LOG_MSG("Error: " << rsp.GetError());
 
                     diag.AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -229,11 +230,10 @@ namespace ignite
 
                 for (size_t i = 0; i < meta.size(); ++i)
                 {
-                    LOG_MSG("[%d] CatalogName: %s\n", i, meta[i].GetCatalogName().c_str());
-                    LOG_MSG("[%d] SchemaName:  %s\n", i, meta[i].GetSchemaName().c_str());
-                    LOG_MSG("[%d] TableName:   %s\n", i, meta[i].GetTableName().c_str());
-                    LOG_MSG("[%d] TableType:   %s\n", i, meta[i].GetTableType().c_str());
-                    LOG_MSG("\n");
+                    LOG_MSG("\n[" << i << "] CatalogName: " << meta[i].GetCatalogName()
+                         << "\n[" << i << "] SchemaName:  " << meta[i].GetSchemaName()
+                         << "\n[" << i << "] TableName:   " << meta[i].GetTableName()
+                         << "\n[" << i << "] TableType:   " << meta[i].GetTableType());
                 }
 
                 return SQL_RESULT_SUCCESS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/statement.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/statement.cpp b/modules/platforms/cpp/odbc/src/statement.cpp
index 09ad81c..2395d66 100644
--- a/modules/platforms/cpp/odbc/src/statement.cpp
+++ b/modules/platforms/cpp/odbc/src/statement.cpp
@@ -27,6 +27,7 @@
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/message.h"
 #include "ignite/odbc/statement.h"
+#include "ignite/odbc/log.h"
 
 namespace ignite
 {
@@ -232,7 +233,7 @@ namespace ignite
                 {
                     SQLULEN val = reinterpret_cast<SQLULEN>(value);
 
-                    LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: %d\n", val);
+                    LOG_MSG("SQL_ATTR_ROW_ARRAY_SIZE: " << val);
 
                     if (val != 1)
                     {
@@ -986,7 +987,7 @@ namespace ignite
             if (paramNum > 0 && static_cast<size_t>(paramNum) <= paramTypes.size())
                 type = paramTypes[paramNum - 1];
 
-            LOG_MSG("Type: %d\n", type);
+            LOG_MSG("Type: " << type);
 
             if (!type)
             {
@@ -1044,7 +1045,7 @@ namespace ignite
 
             if (rsp.GetStatus() != RESPONSE_STATUS_SUCCESS)
             {
-                LOG_MSG("Error: %s\n", rsp.GetError().c_str());
+                LOG_MSG("Error: " << rsp.GetError());
 
                 AddStatusRecord(SQL_STATE_HY000_GENERAL_ERROR, rsp.GetError());
 
@@ -1054,7 +1055,9 @@ namespace ignite
             paramTypes = rsp.GetTypeIds();
 
             for (size_t i = 0; i < paramTypes.size(); ++i)
-                LOG_MSG("[%zu] Parameter type: %u\n", i, paramTypes[i]);
+            {
+                LOG_MSG("[" << i << "] Parameter type: " << paramTypes[i]);
+            }
 
             return SQL_RESULT_SUCCESS;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/825fe9cd/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index c15c23c..63454dc 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -22,20 +22,6 @@
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/system/odbc_constants.h"
 
-#ifdef ODBC_DEBUG
-
-FILE* log_file = NULL;
-
-void logInit(const char* path)
-{
-    if (!log_file)
-    {
-        log_file = fopen(path, "w");
-    }
-}
-
-#endif //ODBC_DEBUG
-
 namespace ignite
 {
     namespace utility
@@ -155,11 +141,11 @@ namespace ignite
                 res.clear();
         }
 
-        std::string HexDump(const char* data, size_t count)
+        std::string HexDump(const void* data, size_t count)
         {
             std::stringstream  dump;
             size_t cnt = 0;
-            for(const char* p = data, *e = data + count; p != e; ++p)
+            for(const uint8_t* p = (const uint8_t*)data, *e = (const uint8_t*)data + count; p != e; ++p)
             {
                 if (cnt++ % 16 == 0)
                 {


[43/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
index 5371856..cf67041 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteDistributedJoinTestSuite.java
@@ -34,7 +34,7 @@ import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistr
  */
 public class IgniteDistributedJoinTestSuite extends TestSuite {
     /**
-     *
+     * @return Suite.
      */
     public static TestSuite suite() {
         TestSuite suite = new TestSuite("Distributed Joins Test Suite.");


[38/50] [abbrv] ignite git commit: Merge branch ignite-gg-8.0.2.ea2 into ignite-db-x

Posted by sb...@apache.org.
Merge branch ignite-gg-8.0.2.ea2 into ignite-db-x


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 0ee73fb097a6d0e15fbfd6ae8acedac46c45a640
Parents: 53e2cd2 b0ac987
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 17 13:06:46 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 17 13:06:46 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |    1 +
 README.md                                       |    8 +
 RELEASE_NOTES.txt                               |   22 +
 assembly/dependencies-fabric-lgpl.xml           |    1 +
 assembly/dependencies-fabric.xml                |    1 +
 assembly/release-base.xml                       |    5 +
 assembly/release-fabric-base.xml                |    1 +
 bin/control.bat                                 |  229 ++
 bin/control.sh                                  |  180 +
 bin/ignite.sh                                   |    9 +
 config/dotnet/default-dotnet.xml                |   45 +
 examples/pom-standalone.xml                     |    6 +
 examples/pom.xml                                |    2 +-
 examples/redis/redis-example.php                |   82 +
 examples/redis/redis-example.py                 |   62 +
 examples/schema-import/pom.xml                  |    2 +-
 .../ignite/examples/ExampleNodeStartup.java     |    2 +-
 .../examples/datagrid/CachePutGetExample.java   |    2 +-
 .../examples/datagrid/CacheQueryDmlExample.java |  163 +
 .../scalar/examples/ScalarJvmCloudExample.scala |    2 +-
 .../CacheExamplesMultiNodeSelfTest.java         |    7 +
 .../ignite/examples/CacheExamplesSelfTest.java  |    8 +
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/benchmarks/pom.xml                      |    2 +-
 .../jmh/misc/JmhIncrementBenchmark.java         |   95 +
 modules/camel/pom.xml                           |    2 +-
 modules/cassandra/pom.xml                       |   16 +-
 modules/cassandra/serializers/README.txt        |    7 +-
 modules/cassandra/serializers/pom.xml           |    4 +-
 modules/cassandra/store/pom.xml                 |    4 +-
 .../store/cassandra/CassandraCacheStore.java    |    6 +
 .../store/cassandra/datasource/DataSource.java  |    9 +
 modules/clients/pom.xml                         |   12 +-
 .../clients/src/test/config/jdbc-bin-config.xml |   54 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |  263 ++
 .../JdbcAbstractUpdateStatementSelfTest.java    |   37 +
 ...BinaryMarshallerInsertStatementSelfTest.java |   37 +
 ...cBinaryMarshallerMergeStatementSelfTest.java |   37 +
 .../jdbc2/JdbcDeleteStatementSelfTest.java      |   49 +
 .../jdbc2/JdbcInsertStatementSelfTest.java      |  122 +
 .../jdbc2/JdbcMergeStatementSelfTest.java       |   91 +
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |   40 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |   18 +
 .../rest/ChangeStateCommandHandlerTest.java     |  125 +
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |   11 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |   48 +-
 .../rest/protocols/tcp/MockNioSession.java      |   11 +
 .../tcp/redis/RedisProtocolSelfTest.java        |  101 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    5 +
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |    2 +
 modules/core/pom.xml                            |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |    8 +-
 .../java/org/apache/ignite/IgniteCache.java     |    9 +
 .../apache/ignite/IgniteCheckedException.java   |   12 +
 .../java/org/apache/ignite/IgniteLogger.java    |    6 +-
 .../java/org/apache/ignite/IgniteServices.java  |   16 +
 .../apache/ignite/IgniteSystemProperties.java   |   58 +
 .../binary/BinaryAbstractIdentityResolver.java  |   53 +
 .../binary/BinaryArrayIdentityResolver.java     |  224 ++
 .../binary/BinaryFieldIdentityResolver.java     |  307 ++
 .../ignite/binary/BinaryIdentityResolver.java   |   42 +
 .../ignite/binary/BinaryTypeConfiguration.java  |   27 +-
 .../org/apache/ignite/cache/QueryEntity.java    |   27 +
 .../rendezvous/RendezvousAffinityFunction.java  |   80 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |    2 +-
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |    6 +-
 .../ignite/cache/store/CacheStoreAdapter.java   |    6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |  112 +-
 .../store/jdbc/dialect/BasicJdbcDialect.java    |   31 +-
 .../configuration/CacheConfiguration.java       |   49 +-
 .../configuration/IgniteConfiguration.java      |   50 +-
 .../apache/ignite/internal/GridComponent.java   |    7 +-
 .../internal/GridEventConsumeHandler.java       |    7 +-
 .../ignite/internal/GridJobContextImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |   17 +
 .../ignite/internal/GridKernalContextImpl.java  |   28 +-
 .../internal/GridMessageListenHandler.java      |    5 +-
 .../internal/GridPerformanceSuggestions.java    |    2 +-
 .../ignite/internal/GridPluginComponent.java    |    4 +-
 .../org/apache/ignite/internal/GridTopic.java   |    6 +-
 .../ignite/internal/IgniteInternalFuture.java   |   11 +
 .../apache/ignite/internal/IgniteKernal.java    |  187 +-
 .../ignite/internal/IgniteServicesImpl.java     |    9 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   37 +-
 .../ignite/internal/MarshallerContextImpl.java  |   10 +-
 .../internal/binary/BinaryClassDescriptor.java  |   27 +-
 .../ignite/internal/binary/BinaryContext.java   |   80 +-
 .../ignite/internal/binary/BinaryFieldImpl.java |   10 +-
 .../internal/binary/BinaryObjectExImpl.java     |   90 +-
 .../internal/binary/BinaryObjectImpl.java       |   48 +-
 .../binary/BinaryObjectOffheapImpl.java         |   42 +-
 .../internal/binary/BinaryPrimitives.java       |   30 +-
 .../binary/BinarySerializedFieldComparator.java |  343 ++
 .../ignite/internal/binary/BinaryTypeProxy.java |   15 +-
 .../ignite/internal/binary/BinaryUtils.java     |   16 +-
 .../internal/binary/BinaryWriterExImpl.java     |   47 +-
 .../internal/binary/GridBinaryMarshaller.java   |    2 +-
 .../binary/builder/BinaryBuilderReader.java     |   11 +
 .../binary/builder/BinaryBuilderSerializer.java |    4 +
 .../binary/builder/BinaryObjectBuilderImpl.java |   27 +-
 .../streams/BinaryAbstractInputStream.java      |    5 +
 .../streams/BinaryAbstractOutputStream.java     |    5 +
 .../streams/BinaryByteBufferInputStream.java    |   10 +
 .../binary/streams/BinaryHeapInputStream.java   |    9 +-
 .../binary/streams/BinaryHeapOutputStream.java  |    7 +-
 .../streams/BinaryMemoryAllocatorChunk.java     |    3 +-
 .../streams/BinaryOffheapInputStream.java       |   14 +-
 .../streams/BinaryOffheapOutputStream.java      |    8 +-
 .../internal/binary/streams/BinaryStream.java   |   12 +-
 .../ignite/internal/client/GridClient.java      |    9 +
 .../internal/client/GridClientClusterState.java |   33 +
 .../client/impl/GridClientClusterStateImpl.java |   75 +
 .../internal/client/impl/GridClientImpl.java    |   11 +
 .../impl/connection/GridClientConnection.java   |   17 +
 .../GridClientConnectionManagerAdapter.java     |    8 +-
 .../connection/GridClientNioTcpConnection.java  |   21 +
 .../impl/connection/GridClientTopology.java     |   53 +-
 .../router/impl/GridRouterClientImpl.java       |    6 +
 .../internal/commandline/CommandHandler.java    |   89 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |   17 +
 .../internal/jdbc2/JdbcPreparedStatement.java   |   71 +-
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |    5 +-
 .../ignite/internal/jdbc2/JdbcQueryTaskV2.java  |  406 ++
 .../ignite/internal/jdbc2/JdbcResultSet.java    |   72 +-
 .../internal/jdbc2/JdbcSqlFieldsQuery.java      |   49 +
 .../ignite/internal/jdbc2/JdbcStatement.java    |  213 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |   25 +-
 .../internal/managers/GridManagerAdapter.java   |    2 +-
 .../checkpoint/GridCheckpointManager.java       |    2 +-
 .../collision/GridCollisionManager.java         |    2 +-
 .../managers/communication/GridIoManager.java   |   35 +-
 .../managers/communication/GridIoMessage.java   |   13 +
 .../communication/GridIoMessageFactory.java     |   71 +-
 .../communication/IgniteIoTestMessage.java      |   13 +-
 .../deployment/GridDeploymentManager.java       |    2 +-
 .../discovery/GridDiscoveryManager.java         |  125 +-
 .../eventstorage/GridEventStorageManager.java   |    2 +-
 .../managers/failover/GridFailoverManager.java  |    2 +-
 .../managers/indexing/GridIndexingManager.java  |    2 +-
 .../loadbalancer/GridLoadBalancerManager.java   |    2 +-
 .../ignite/internal/pagemem/FullPageId.java     |   30 +-
 .../ignite/internal/pagemem/PageIdUtils.java    |    8 +
 .../pagemem/backup/BackupFinishedMessage.java   |  136 -
 .../StartFullBackupAckDiscoveryMessage.java     |  128 -
 .../backup/StartFullBackupDiscoveryMessage.java |  135 -
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |    2 +-
 .../snapshot/SnapshotFinishedMessage.java       |  136 +
 .../snapshot/SnapshotProgressMessage.java       |  135 +
 .../StartFullSnapshotAckDiscoveryMessage.java   |  147 +
 .../StartFullSnapshotDiscoveryMessage.java      |  160 +
 .../pagemem/store/IgnitePageStoreManager.java   |   13 +-
 .../internal/pagemem/store/PageStore.java       |    2 +-
 .../pagemem/wal/IgniteWriteAheadLogManager.java |    3 +-
 .../wal/record/MemoryRecoveryRecord.java        |   23 +-
 .../internal/pagemem/wal/record/WALRecord.java  |   23 +-
 .../MetaPageUpdateCandidateAllocatedIndex.java  |   63 -
 .../MetaPageUpdateLastSuccessfulBackupId.java   |   59 -
 ...etaPageUpdateLastSuccessfulFullBackupId.java |   59 -
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |   59 +
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |   70 +
 .../delta/MetaPageUpdateNextBackupId.java       |   59 -
 .../delta/MetaPageUpdateNextSnapshotId.java     |   59 +
 .../MetaPageUpdatePartitionDataRecord.java      |   15 +-
 .../delta/PageListMetaResetCountRecord.java     |   47 +
 .../record/delta/TrackingPageDeltaRecord.java   |   28 +-
 .../processors/GridProcessorAdapter.java        |    4 +-
 .../affinity/GridAffinityProcessor.java         |    4 +-
 .../cache/CacheAffinitySharedManager.java       |    9 +
 .../processors/cache/CacheLockCandidates.java   |   42 +
 .../cache/CacheLockCandidatesList.java          |   71 +
 .../internal/processors/cache/CacheState.java   |   33 -
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../cache/ChangeGlobalStateMessage.java         |  120 +
 .../internal/processors/cache/ClusterState.java |   38 +
 .../cache/DynamicCacheChangeRequest.java        |   31 +-
 .../cache/DynamicCacheDescriptor.java           |   17 +
 .../processors/cache/GridCacheAdapter.java      |  756 ++--
 .../processors/cache/GridCacheContext.java      |    8 +
 .../processors/cache/GridCacheEntryEx.java      |    7 +-
 .../cache/GridCacheLoaderWriterStore.java       |    6 +
 .../processors/cache/GridCacheMapEntry.java     |  141 +-
 .../processors/cache/GridCacheMessage.java      |    7 +
 .../processors/cache/GridCacheMvcc.java         |  376 +-
 .../processors/cache/GridCacheMvccCallback.java |    4 +-
 .../cache/GridCacheMvccCandidate.java           |   80 +-
 .../processors/cache/GridCacheMvccManager.java  |   19 +-
 .../GridCachePartitionExchangeManager.java      |  211 +-
 .../processors/cache/GridCachePreloader.java    |   16 +-
 .../cache/GridCachePreloaderAdapter.java        |   14 +-
 .../processors/cache/GridCacheProcessor.java    |  472 ++-
 .../processors/cache/GridCacheProxyImpl.java    |   24 +-
 .../processors/cache/GridCacheTtlManager.java   |    9 +-
 .../processors/cache/GridCacheUtils.java        |   62 +-
 .../GridChangeGlobalStateMessageResponse.java   |  177 +
 .../cache/IgniteCacheOffheapManager.java        |   22 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  184 +-
 .../processors/cache/IgniteCacheProxy.java      |   14 +-
 .../processors/cache/IgniteInternalCache.java   |    8 +
 .../processors/cache/QueryCursorImpl.java       |   31 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |    9 +-
 .../IgniteCacheDatabaseSharedManager.java       |   66 +-
 .../cache/database/freelist/PagesList.java      |    4 +
 .../cache/database/tree/BPlusTree.java          |   23 +-
 .../cache/database/tree/io/PageMetaIO.java      |   95 +-
 .../cache/database/tree/io/TrackingPageIO.java  |  124 +-
 .../cache/database/tree/util/PageHandler.java   |   15 +-
 .../CacheDataStructuresManager.java             |    6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |    4 +-
 .../distributed/GridDistributedCacheEntry.java  |  303 +-
 .../distributed/GridDistributedLockRequest.java |    5 +
 .../GridDistributedTxRemoteAdapter.java         |   14 +-
 .../GridDistributedUnlockRequest.java           |    5 +
 .../dht/GridClientPartitionTopology.java        |   71 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   34 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  129 +-
 .../distributed/dht/GridDhtLockFuture.java      |   61 +-
 .../distributed/dht/GridDhtLockResponse.java    |    9 +-
 .../dht/GridDhtPartitionTopology.java           |   22 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |  382 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |    1 -
 .../distributed/dht/GridDhtTxFinishFuture.java  |   79 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  125 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    7 +-
 .../dht/GridDhtTxOnePhaseCommitAckRequest.java  |    2 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  461 ++-
 .../GridDhtAtomicSingleUpdateRequest.java       |    7 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    5 +
 ...idNearAtomicAbstractSingleUpdateRequest.java |   28 +-
 .../atomic/GridNearAtomicFullUpdateRequest.java |   69 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |    6 +-
 .../GridNearAtomicSingleUpdateRequest.java      |    7 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   13 -
 .../colocated/GridDhtColocatedLockFuture.java   |   12 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  277 +-
 .../dht/preloader/GridDhtPartitionFullMap.java  |   18 +-
 .../dht/preloader/GridDhtPartitionMap2.java     |    4 +
 .../GridDhtPartitionsAbstractMessage.java       |    3 +-
 .../GridDhtPartitionsExchangeFuture.java        |  218 +-
 .../preloader/GridDhtPartitionsFullMessage.java |   83 +-
 .../GridDhtPartitionsSingleMessage.java         |   81 +-
 .../GridDhtPartitionsSingleRequest.java         |    3 +-
 .../dht/preloader/GridDhtPreloader.java         |   47 +-
 .../distributed/near/GridNearAtomicCache.java   |   65 +-
 .../distributed/near/GridNearCacheEntry.java    |   44 +-
 .../distributed/near/GridNearGetRequest.java    |    5 +
 .../distributed/near/GridNearLockFuture.java    |    7 +-
 ...arOptimisticSerializableTxPrepareFuture.java |    4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    4 +-
 .../near/GridNearTransactionalCache.java        |    5 +-
 .../near/GridNearTxFinishFuture.java            |   46 +-
 .../cache/distributed/near/GridNearTxLocal.java |   48 +-
 .../processors/cache/local/GridLocalCache.java  |    7 +-
 .../cache/local/GridLocalCacheEntry.java        |  173 +-
 .../cache/local/GridLocalLockFuture.java        |    2 +-
 .../local/atomic/GridLocalAtomicCache.java      |  174 +-
 .../query/GridCacheDistributedQueryManager.java |    2 +-
 .../cache/query/GridCacheQueryManager.java      |   48 +-
 .../cache/query/GridCacheQueryRequest.java      |    6 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  253 --
 .../cache/query/IgniteQueryErrorCode.java       |   91 +
 .../continuous/CacheContinuousQueryHandler.java |   37 +-
 .../cache/store/CacheStoreManager.java          |    4 +-
 .../store/GridCacheStoreManagerAdapter.java     |   34 +-
 .../cache/store/GridCacheWriteBehindStore.java  |    6 +-
 .../cache/transactions/IgniteTxHandler.java     |   17 +-
 .../transactions/IgniteTxLocalAdapter.java      |   38 +-
 .../cache/transactions/IgniteTxManager.java     |   20 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    2 +-
 .../clock/GridClockSyncProcessor.java           |    8 +-
 .../closure/GridClosureProcessor.java           |   33 +-
 .../processors/cluster/ClusterProcessor.java    |    2 +-
 .../cluster/GridClusterStateProcessor.java      |  941 +++++
 .../cluster/IgniteChangeGlobalStateSupport.java |   40 +
 .../continuous/GridContinuousHandler.java       |    5 +-
 .../continuous/GridContinuousProcessor.java     |    7 +-
 .../StartRoutineAckDiscoveryMessage.java        |   13 +-
 .../StartRoutineDiscoveryMessage.java           |   19 +-
 .../datastreamer/DataStreamProcessor.java       |   24 +-
 .../datastructures/DataStructuresProcessor.java |  132 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   14 +-
 .../GridCacheAtomicReferenceImpl.java           |   16 +-
 .../GridCacheAtomicSequenceImpl.java            |   26 +-
 .../GridCacheAtomicStampedImpl.java             |   14 +-
 .../GridCacheCountDownLatchImpl.java            |   14 +-
 .../datastructures/GridCacheLockImpl.java       |   17 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |   14 +-
 .../processors/hadoop/HadoopClassLoader.java    |   11 +
 .../internal/processors/hadoop/HadoopJobId.java |   79 +-
 .../processors/hadoop/HadoopJobProperty.java    |   64 +-
 .../hadoop/HadoopMapperAwareTaskOutput.java     |   32 +
 .../processors/hadoop/HadoopTaskContext.java    |    8 +
 .../processors/hadoop/HadoopTaskInfo.java       |   43 +
 .../io/PartiallyOffheapRawComparatorEx.java     |   33 +
 .../hadoop/message/HadoopMessage.java           |   27 +
 .../shuffle/HadoopDirectShuffleMessage.java     |  243 ++
 .../hadoop/shuffle/HadoopShuffleAck.java        |  170 +
 .../shuffle/HadoopShuffleFinishRequest.java     |  172 +
 .../shuffle/HadoopShuffleFinishResponse.java    |  142 +
 .../hadoop/shuffle/HadoopShuffleMessage.java    |  361 ++
 .../internal/processors/igfs/IgfsContext.java   |    4 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |    4 +-
 .../processors/job/GridJobProcessor.java        |    2 +-
 .../internal/processors/job/GridJobWorker.java  |   76 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    2 +-
 .../processors/odbc/OdbcMessageParser.java      |   16 +
 .../processors/odbc/OdbcNioListener.java        |    2 +-
 .../internal/processors/odbc/OdbcProcessor.java |   11 +-
 .../odbc/OdbcQueryGetParamsMetaRequest.java     |   60 +
 .../odbc/OdbcQueryGetParamsMetaResult.java      |   40 +
 .../internal/processors/odbc/OdbcRequest.java   |    3 +
 .../processors/odbc/OdbcRequestHandler.java     |  181 +-
 .../internal/processors/odbc/OdbcTableMeta.java |   15 +-
 .../internal/processors/odbc/OdbcUtils.java     |    4 +-
 .../platform/PlatformAbstractTarget.java        |  268 +-
 .../platform/PlatformAsyncTarget.java           |   44 +
 .../platform/PlatformContextImpl.java           |    2 +-
 .../platform/PlatformNoopProcessor.java         |   41 +-
 .../processors/platform/PlatformProcessor.java  |   42 +-
 .../platform/PlatformProcessorImpl.java         |   89 +-
 .../processors/platform/PlatformTarget.java     |  103 +-
 .../platform/PlatformTargetProxy.java           |  126 +
 .../platform/PlatformTargetProxyImpl.java       |  222 ++
 .../binary/PlatformBinaryProcessor.java         |    6 +-
 .../platform/cache/PlatformCache.java           |   15 +-
 .../cache/PlatformCacheEntryFilterImpl.java     |    4 +-
 .../cache/PlatformCacheEntryProcessorImpl.java  |   43 +-
 .../platform/cache/PlatformCacheIterator.java   |    2 +-
 .../cache/affinity/PlatformAffinity.java        |    4 +-
 .../affinity/PlatformAffinityFunction.java      |   59 +-
 .../PlatformAffinityFunctionTarget.java         |    4 +-
 .../query/PlatformAbstractQueryCursor.java      |    4 +-
 .../query/PlatformContinuousQueryProxy.java     |    3 +-
 .../callback/PlatformCallbackGateway.java       |  272 +-
 .../platform/callback/PlatformCallbackOp.java   |  206 +
 .../callback/PlatformCallbackUtils.java         |  544 +--
 .../platform/cluster/PlatformClusterGroup.java  |   18 +-
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../platform/compute/PlatformAbstractTask.java  |   15 +-
 .../platform/compute/PlatformClosureJob.java    |   12 +-
 .../platform/compute/PlatformCompute.java       |   21 +-
 .../platform/compute/PlatformFullJob.java       |   15 +-
 .../platform/compute/PlatformFullTask.java      |   18 +-
 .../datastreamer/PlatformDataStreamer.java      |    4 +-
 .../PlatformStreamReceiverImpl.java             |   11 +-
 .../datastructures/PlatformAtomicLong.java      |    4 +-
 .../datastructures/PlatformAtomicReference.java |    8 +-
 .../datastructures/PlatformAtomicSequence.java  |    2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   17 +-
 .../platform/events/PlatformEvents.java         |   15 +-
 .../memory/PlatformInputStreamImpl.java         |   14 +-
 .../memory/PlatformOutputStreamImpl.java        |   14 +-
 .../platform/messaging/PlatformMessaging.java   |    9 +-
 .../services/PlatformAbstractService.java       |   25 +-
 .../platform/services/PlatformServices.java     |   29 +-
 .../transactions/PlatformTransactions.java      |    8 +-
 .../platform/utils/PlatformFutureUtils.java     |   18 +-
 .../utils/PlatformListenableTarget.java         |   62 +
 .../platform/utils/PlatformUtils.java           |   36 +-
 .../plugin/IgnitePluginProcessor.java           |    2 +-
 .../processors/port/GridPortProcessor.java      |    2 +-
 .../processors/query/GridQueryIndexing.java     |   42 +-
 .../processors/query/GridQueryProcessor.java    |  646 +++-
 .../processors/query/GridQueryProperty.java     |   20 +
 .../query/GridQueryTypeDescriptor.java          |   25 +
 .../processors/query/IgniteSQLException.java    |   89 +
 .../resource/GridResourceProcessor.java         |    2 +-
 .../processors/rest/GridRestCommand.java        |   11 +-
 .../processors/rest/GridRestProcessor.java      |   22 +-
 .../client/message/GridClientStateRequest.java  |   80 +
 .../cluster/GridChangeStateCommandHandler.java  |   89 +
 .../redis/GridRedisRestCommandHandler.java      |    1 +
 .../redis/key/GridRedisDelCommandHandler.java   |    7 +-
 .../key/GridRedisExistsCommandHandler.java      |    7 +-
 .../server/GridRedisDbSizeCommandHandler.java   |    7 +-
 .../string/GridRedisAppendCommandHandler.java   |    7 +-
 .../string/GridRedisGetCommandHandler.java      |   34 +-
 .../string/GridRedisGetRangeCommandHandler.java |    7 +-
 .../string/GridRedisGetSetCommandHandler.java   |    7 +-
 .../string/GridRedisIncrDecrCommandHandler.java |   70 +-
 .../string/GridRedisMGetCommandHandler.java     |    7 +-
 .../string/GridRedisMSetCommandHandler.java     |    7 +-
 .../string/GridRedisSetCommandHandler.java      |   31 +-
 .../string/GridRedisSetRangeCommandHandler.java |    7 +-
 .../string/GridRedisStrlenCommandHandler.java   |    7 +-
 .../tcp/GridTcpMemcachedNioListener.java        |   15 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   22 +-
 .../tcp/redis/GridRedisNioListener.java         |    4 +-
 .../request/GridRestChangeStateRequest.java     |   57 +
 .../service/GridServiceProcessor.java           |  144 +-
 .../processors/service/GridServiceProxy.java    |   18 +-
 .../session/GridTaskSessionProcessor.java       |    2 +-
 .../processors/task/GridTaskProcessor.java      |    6 +-
 .../processors/task/GridTaskWorker.java         |    7 +
 .../timeout/GridTimeoutProcessor.java           |    2 +-
 .../util/GridCursorIteratorWrapper.java         |   36 +
 .../ignite/internal/util/GridHandleTable.java   |   10 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   70 +-
 .../apache/ignite/internal/util/IgniteTree.java |   70 +
 .../ignite/internal/util/IgniteUtils.java       |  121 +-
 .../internal/util/OffheapReadWriteLock.java     |    4 +
 .../ignite/internal/util/StripedExecutor.java   |  667 ++++
 .../util/future/GridCompoundFuture.java         |   56 +-
 .../util/future/GridFinishedFuture.java         |   24 +
 .../internal/util/future/GridFutureAdapter.java |   15 +-
 .../util/future/GridFutureChainListener.java    |   30 +-
 .../util/future/IgniteRemoteMapTask.java        |   17 +
 .../internal/util/io/GridUnsafeDataInput.java   |   12 +-
 .../internal/util/io/GridUnsafeDataOutput.java  |   12 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    2 +-
 .../util/lang/IgniteSingletonIterator.java      |   56 +
 .../util/nio/GridCommunicationClient.java       |    4 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   15 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   10 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   17 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   16 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   10 +-
 .../internal/util/nio/GridNioFilterChain.java   |   14 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    4 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   95 +-
 .../ignite/internal/util/nio/GridNioServer.java | 1343 +++++--
 .../internal/util/nio/GridNioSession.java       |   11 +
 .../internal/util/nio/GridNioSessionImpl.java   |   49 +-
 .../ignite/internal/util/nio/GridNioWorker.java |   48 +
 .../util/nio/GridSelectorNioSessionImpl.java    |  157 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   49 +-
 .../internal/util/nio/SessionWriteRequest.java  |   85 +
 .../internal/util/nio/ssl/GridNioSslFilter.java |   10 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    4 +-
 .../offheap/unsafe/GridOffHeapSnapTreeMap.java  |   11 +-
 .../util/offheap/unsafe/GridUnsafeMemory.java   |   10 +-
 .../internal/util/snaptree/SnapTreeMap.java     |   10 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../internal/visor/VisorMultiNodeTask.java      |    2 +-
 .../internal/visor/VisorTaskArgument.java       |    2 +-
 .../ignite/internal/visor/cache/VisorCache.java |   87 +-
 .../visor/cache/VisorCachePartition.java        |   14 +-
 .../visor/cache/VisorCachePartitions.java       |   10 +-
 .../visor/cache/VisorCachePartitionsTask.java   |    4 +-
 .../visor/cache/VisorCacheSwapBackupsTask.java  |  102 -
 .../visor/event/VisorGridDiscoveryEvent.java    |   18 +-
 .../visor/event/VisorGridDiscoveryEventV2.java  |   80 -
 .../misc/VisorChangeGridActiveStateTask.java    |   65 +
 .../visor/node/VisorNodeDataCollectorJob.java   |    3 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    2 +
 .../node/VisorNodeDataCollectorTaskResult.java  |   17 +
 .../internal/visor/query/VisorQueryJob.java     |    2 +-
 .../visor/service/VisorCancelServiceTask.java   |   70 +
 .../visor/service/VisorServiceDescriptor.java   |  132 +
 .../visor/service/VisorServiceTask.java         |   75 +
 .../internal/visor/util/VisorEventMapper.java   |    4 +-
 .../internal/visor/util/VisorTaskUtils.java     |   38 +-
 .../apache/ignite/logger/java/JavaLogger.java   |    4 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |    4 +-
 .../optimized/OptimizedMarshaller.java          |    8 +-
 .../optimized/OptimizedObjectOutputStream.java  |   25 +-
 .../org/apache/ignite/mxbean/IgniteMXBean.java  |   21 +
 .../communication/tcp/TcpCommunicationSpi.java  |  324 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   40 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   74 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   22 +-
 .../tcp/internal/TcpDiscoveryNode.java          |    5 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |    4 +
 .../org/apache/ignite/stream/StreamAdapter.java |    4 +-
 .../ignite/thread/IgniteThreadFactory.java      |    8 +-
 .../apache/ignite/util/AttributeNodeFilter.java |  108 +
 .../org/jsr166/ConcurrentLinkedHashMap.java     |    2 +-
 .../resources/META-INF/classnames.properties    |  172 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../AbstractAffinityFunctionSelfTest.java       |    2 +-
 .../store/jdbc/CacheJdbcPojoStoreTest.java      |   48 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   |  283 ++
 .../internal/ClusterNodeMetricsSelfTest.java    |    2 +
 .../IgniteComputeTopologyExceptionTest.java     |    5 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |  300 ++
 .../internal/binary/BinaryEnumsSelfTest.java    |   18 +
 .../BinaryFieldIdentityResolverSelfTest.java    |  333 ++
 .../binary/BinaryFieldsOffheapSelfTest.java     |    2 +-
 .../BinaryFooterOffsetsOffheapSelfTest.java     |    2 +-
 ...ryIdentityResolverConfigurationSelfTest.java |  138 +
 .../binary/BinaryMarshallerSelfTest.java        |   68 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |  157 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java |    2 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |  568 +++
 ...GridManagerLocalMessageListenerSelfTest.java |    8 +-
 .../managers/GridNoopManagerSelfTest.java       |    2 +-
 .../GridCommunicationSendMessageSelfTest.java   |    7 +-
 ...unicationBalanceMultipleConnectionsTest.java |   28 +
 .../IgniteCommunicationBalanceTest.java         |  339 ++
 .../communication/IgniteIoTestMessagesTest.java |   95 +
 .../IgniteVariousConnectionNumberTest.java      |  166 +
 .../GridDeploymentMessageCountSelfTest.java     |    3 +
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    2 +-
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |    2 +-
 .../cache/CacheClientStoreSelfTest.java         |    6 +-
 .../cache/CacheConfigurationLeakTest.java       |   15 +
 .../cache/CacheEntryProcessorCopySelfTest.java  |    8 -
 .../cache/CacheRebalancingSelfTest.java         |   75 +
 .../CacheSerializableTransactionsTest.java      |  604 ++-
 .../cache/ClusterStateAbstractTest.java         |   46 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |   30 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    6 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   11 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |    8 +-
 .../cache/GridCacheAbstractTxReadTest.java      |    2 -
 .../cache/GridCacheBasicStoreAbstractTest.java  |    7 +-
 ...GridCacheConcurrentGetCacheOnClientTest.java |  129 +
 .../GridCacheMissingCommitVersionSelfTest.java  |    6 +-
 .../cache/GridCacheMvccFlagsTest.java           |    8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |  334 +-
 .../processors/cache/GridCacheMvccSelfTest.java |  212 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |    6 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |    4 +
 .../cache/GridCacheP2PUndeploySelfTest.java     |    5 -
 .../GridCachePartitionedAffinitySpreadTest.java |    7 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   80 +-
 .../GridCacheTtlManagerEvictionSelfTest.java    |    1 +
 ...idCacheValueConsistencyAbstractSelfTest.java |   30 -
 .../IgniteCacheConfigVariationsFullApiTest.java |   35 +-
 .../IgniteCacheInterceptorSelfTestSuite.java    |   10 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |    6 +
 .../IgniteCacheReadThroughEvictionSelfTest.java |    5 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    1 +
 ...heapCacheMetricsForClusterGroupSelfTest.java |  141 +
 .../GridCacheBinaryObjectsAbstractSelfTest.java |  260 +-
 .../GridDataStreamerImplSelfTest.java           |    8 +-
 .../database/tree/io/TrackingPageIOTest.java    |   21 +-
 ...achePartitionedAtomicSequenceTxSelfTest.java |  169 +
 ...CacheLoadingConcurrentGridStartSelfTest.java |   20 +-
 .../CacheLockReleaseNodeLeaveTest.java          |  135 +
 ...tractDistributedByteArrayValuesSelfTest.java |    3 +
 ...eAtomicMessageRecovery10ConnectionsTest.java |   28 +
 ...cMessageRecoveryNoPairedConnectionsTest.java |   47 +
 ...acheConnectionRecovery10ConnectionsTest.java |   35 +
 .../distributed/IgniteCacheCreatePutTest.java   |    2 +-
 .../IgniteTxCachePrimarySyncTest.java           |    7 +
 ...teSynchronizationModesMultithreadedTest.java |    5 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |  232 --
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   13 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   16 +-
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    2 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +-
 .../near/GridCacheNearReadersSelfTest.java      |    3 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   37 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 .../near/NoneRebalanceModeSelfTest.java         |    4 +-
 .../GridCacheRebalancingSyncSelfTest.java       |    2 +
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |    3 +
 ...maryWriteOrderWithStoreExpiryPolicyTest.java |    2 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    6 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |   57 +-
 .../CacheOffHeapAndSwapMetricsSelfTest.java     |  617 ---
 ...cheLocalBasicStoreMultithreadedSelfTest.java |    1 +
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  627 +++
 ...ContinuousQueryFailoverAbstractSelfTest.java |    6 +-
 .../CacheKeepBinaryIterationTest.java           |    3 +
 .../GridCacheContinuousQueryConcurrentTest.java |   47 +-
 ...teCacheContinuousQueryNoUnsubscribeTest.java |    2 +
 .../TxDeadlockDetectionNoHangsTest.java         |    2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   29 +-
 .../closure/GridClosureSerializationTest.java   |  177 +
 .../processors/database/BPlusTreeSelfTest.java  |   33 +
 ...lockMessageSystemPoolStarvationSelfTest.java |   14 +-
 .../GridServiceProcessorProxySelfTest.java      |    2 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |  284 ++
 .../util/future/GridFutureAdapterSelfTest.java  |  122 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   18 +-
 .../loadtests/hashmap/GridHashMapLoadTest.java  |    7 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    2 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   37 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   13 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    6 +-
 ...ationSpiRecoveryNoPairedConnectionsTest.java |   28 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    8 +
 ...CommunicationRecoveryAckClosureSelfTest.java |    6 +-
 .../ignite/testframework/GridTestUtils.java     |   80 +-
 .../testframework/junits/GridAbstractTest.java  |    2 +-
 .../junits/GridTestKernalContext.java           |    6 +-
 .../junits/logger/GridTestLog4jLogger.java      |    4 +-
 .../multijvm/IgniteCacheProcessProxy.java       |    5 +
 .../junits/multijvm/IgniteNodeRunner.java       |    2 +-
 .../junits/multijvm/IgniteProcessProxy.java     |    2 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |    5 +
 .../IgniteBinaryObjectsTestSuite.java           |   10 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |    5 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    4 +
 .../IgniteCacheMetricsSelfTestSuite.java        |    6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   29 +-
 .../testsuites/IgniteCacheTestSuite2.java       |    3 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    3 +
 .../testsuites/IgniteCacheTestSuite5.java       |    4 +
 .../testsuites/IgniteComputeGridTestSuite.java  |    1 -
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../IgniteSpiCommunicationSelfTestSuite.java    |    2 +
 .../util/AttributeNodeFilterSelfTest.java       |  184 +
 modules/docker/1.8.0/Dockerfile                 |   46 +
 modules/docker/1.8.0/run.sh                     |   51 +
 modules/docker/Dockerfile                       |    6 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/flink/pom.xml                           |    2 +-
 modules/flume/pom.xml                           |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 .../query/h2/opt/GridH2SpatialIndex.java        |   24 +-
 modules/hadoop/pom.xml                          |    9 +-
 .../hadoop/io/PartiallyRawComparator.java       |   33 +
 .../org/apache/ignite/hadoop/io/RawMemory.java  |   86 +
 .../hadoop/io/TextPartiallyRawComparator.java   |  115 +
 .../apache/ignite/hadoop/io/package-info.java   |   22 +
 .../IgniteHadoopClientProtocolProvider.java     |   70 +-
 .../processors/hadoop/HadoopMapperUtils.java    |   56 +
 .../processors/hadoop/HadoopProcessor.java      |    6 +-
 .../hadoop/impl/fs/HadoopFileSystemsUtils.java  |   11 +
 .../hadoop/impl/proto/HadoopClientProtocol.java |   55 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   11 +
 ...DelegatingPartiallyOffheapRawComparator.java |   54 +
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   22 +-
 .../impl/v2/HadoopV2JobResourceManager.java     |   25 +-
 .../hadoop/impl/v2/HadoopV2MapTask.java         |   10 +
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   21 +
 .../processors/hadoop/io/OffheapRawMemory.java  |  131 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    8 +-
 .../hadoop/mapreduce/MapReduceClient.java       |  147 +
 .../hadoop/message/HadoopMessage.java           |   27 -
 .../hadoop/shuffle/HadoopShuffle.java           |  112 +-
 .../hadoop/shuffle/HadoopShuffleAck.java        |   92 -
 .../hadoop/shuffle/HadoopShuffleJob.java        |  748 +++-
 .../hadoop/shuffle/HadoopShuffleLocalState.java |   67 +
 .../hadoop/shuffle/HadoopShuffleMessage.java    |  242 --
 .../shuffle/HadoopShuffleRemoteState.java       |   61 +
 .../shuffle/collections/HadoopMultimapBase.java |    5 +-
 .../shuffle/collections/HadoopSkipList.java     |   14 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  166 +
 .../shuffle/direct/HadoopDirectDataOutput.java  |  221 ++
 .../direct/HadoopDirectDataOutputContext.java   |  100 +
 .../direct/HadoopDirectDataOutputState.java     |   54 +
 .../shuffle/streams/HadoopDataOutStream.java    |    2 +-
 .../child/HadoopChildProcessRunner.java         |   12 +-
 .../HadoopExternalCommunication.java            |    4 +-
 .../communication/HadoopIpcToNioAdapter.java    |    2 +-
 .../communication/HadoopMarshallerFilter.java   |    6 +-
 .../resources/META-INF/classnames.properties    |  100 +
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   13 +-
 .../impl/HadoopAbstractWordCountTest.java       |    6 +-
 .../hadoop/impl/HadoopFileSystemsTest.java      |    9 +
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    4 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |   22 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |    4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  383 ++
 ...opClientProtocolMultipleServersSelfTest.java |   93 +-
 .../client/HadoopClientProtocolSelfTest.java    |  232 +-
 .../collections/HadoopAbstractMapTest.java      |    6 +
 .../HadoopConcurrentHashMultimapSelftest.java   |    2 +-
 .../collections/HadoopSkipListSelfTest.java     |   16 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |    2 +
 .../testsuites/IgniteHadoopTestSuite.java       |    3 +
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../cache/query/GridCacheTwoStepQuery.java      |  253 ++
 .../query/h2/DmlStatementsProcessor.java        | 1083 ++++++
 .../query/h2/GridH2ResultSetIterator.java       |    3 +-
 .../internal/processors/query/h2/H2Cursor.java  |  105 +
 .../processors/query/h2/IgniteH2Indexing.java   |  383 +-
 .../query/h2/database/H2PkHashIndex.java        |  319 ++
 .../query/h2/database/H2TreeIndex.java          |  132 +-
 .../query/h2/dml/FastUpdateArgument.java        |   27 +
 .../query/h2/dml/FastUpdateArguments.java       |   53 +
 .../query/h2/dml/KeyValueSupplier.java          |   30 +
 .../processors/query/h2/dml/UpdateMode.java     |   36 +
 .../processors/query/h2/dml/UpdatePlan.java     |  128 +
 .../query/h2/dml/UpdatePlanBuilder.java         |  486 +++
 .../processors/query/h2/dml/package-info.java   |   22 +
 .../query/h2/opt/GridH2IndexBase.java           |  182 +-
 .../query/h2/opt/GridH2RowDescriptor.java       |   23 +
 .../processors/query/h2/opt/GridH2Table.java    |  145 +-
 .../query/h2/opt/GridH2TreeIndex.java           |  203 +-
 .../processors/query/h2/sql/DmlAstUtils.java    |  616 +++
 .../processors/query/h2/sql/GridSqlArray.java   |    8 +
 .../processors/query/h2/sql/GridSqlConst.java   |    6 +
 .../processors/query/h2/sql/GridSqlDelete.java  |   68 +
 .../query/h2/sql/GridSqlFunction.java           |    5 +-
 .../processors/query/h2/sql/GridSqlInsert.java  |  149 +
 .../processors/query/h2/sql/GridSqlKeyword.java |   46 +
 .../processors/query/h2/sql/GridSqlMerge.java   |  143 +
 .../processors/query/h2/sql/GridSqlQuery.java   |   44 +-
 .../query/h2/sql/GridSqlQueryParser.java        |  323 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |    6 +-
 .../processors/query/h2/sql/GridSqlSelect.java  |    3 +-
 .../query/h2/sql/GridSqlStatement.java          |   64 +
 .../processors/query/h2/sql/GridSqlUpdate.java  |  105 +
 .../h2/twostep/GridReduceQueryExecutor.java     |   61 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  559 +++
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    2 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |  243 ++
 .../IgniteCacheDeleteSqlQuerySelfTest.java      |  106 +
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  203 +
 .../cache/IgniteCacheLargeResultSelfTest.java   |   15 +-
 ...niteCacheLockPartitionOnAffinityRunTest.java |   36 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  153 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   10 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |  472 +++
 .../IgniteCacheAtomicFieldsQuerySelfTest.java   |   21 -
 ...niteCachePartitionedFieldsQuerySelfTest.java |   25 +
 ...QueryNodeRestartDistributedJoinSelfTest.java |  104 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |    3 +-
 .../query/IgniteSqlDistributedJoinSelfTest.java |  179 +
 .../query/IgniteSqlSplitterSelfTest.java        |    8 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   37 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   37 +-
 .../query/h2/sql/GridQueryParsingTest.java      |  111 +-
 .../IgniteCacheQuerySelfTestSuite.java          |  148 +-
 .../IgniteCacheQuerySelfTestSuite2.java         |  111 +-
 .../IgniteCacheQuerySelfTestSuite3.java         |    2 +
 .../IgniteDistributedJoinTestSuite.java         |   55 +
 .../IgniteH2IndexingSpiTestSuite.java           |   16 +-
 modules/jcl/pom.xml                             |    2 +-
 modules/jms11/pom.xml                           |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |   50 +-
 .../kafka/KafkaIgniteStreamerSelfTest.java      |   36 +-
 modules/log4j/pom.xml                           |    2 +-
 .../apache/ignite/logger/log4j/Log4JLogger.java |    4 +-
 modules/log4j2/pom.xml                          |    2 +-
 modules/mesos/pom.xml                           |    2 +-
 modules/mqtt/pom.xml                            |    2 +-
 modules/osgi-karaf/pom.xml                      |    2 +-
 modules/osgi-paxlogging/pom.xml                 |    2 +-
 modules/osgi/pom.xml                            |    3 +-
 modules/platforms/.gitignore                    |   31 +
 .../include/ignite/binary/binary_raw_reader.h   |   21 +
 .../ignite/impl/binary/binary_reader_impl.h     |    9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |   20 +-
 .../platforms/cpp/common/include/Makefile.am    |    5 +-
 .../common/include/ignite/common/concurrent.h   |   90 +-
 .../include/ignite/common/reference_impl.h      |  286 ++
 .../cpp/common/include/ignite/reference.h       |  564 +++
 .../cpp/common/project/vs/common.vcxproj        |    2 +
 .../common/project/vs/common.vcxproj.filters    |    6 +
 modules/platforms/cpp/configure.ac              |    2 +-
 modules/platforms/cpp/configure.acrel           |    2 +-
 modules/platforms/cpp/core-test/Makefile.am     |    7 +-
 .../core-test/config/cache-query-continuous.xml |   87 +
 .../cpp/core-test/config/cache-query.xml        |   10 +
 .../cpp/core-test/config/cache-test.xml         |    1 +
 .../platforms/cpp/core-test/config/invalid.xml  |   39 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |   13 +-
 .../project/vs/core-test.vcxproj.filters        |   11 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   76 +-
 .../cpp/core-test/src/continuous_query_test.cpp |  611 +++
 .../cpp/core-test/src/handle_registry_test.cpp  |   18 +-
 .../cpp/core-test/src/interop_test.cpp          |   17 +-
 .../cpp/core-test/src/reference_test.cpp        |  412 ++
 modules/platforms/cpp/core/Makefile.am          |    1 +
 .../cpp/core/include/ignite/cache/cache.h       |  104 +-
 .../cpp/core/include/ignite/cache/cache_entry.h |   40 +-
 .../ignite/cache/event/cache_entry_event.h      |  139 +
 .../cache/event/cache_entry_event_listener.h    |   71 +
 .../cache/query/continuous/continuous_query.h   |  239 ++
 .../query/continuous/continuous_query_handle.h  |  133 +
 .../core/include/ignite/impl/cache/cache_impl.h |  116 +-
 .../continuous/continuous_query_handle_impl.h   |  101 +
 .../query/continuous/continuous_query_impl.h    |  351 ++
 .../core/include/ignite/impl/handle_registry.h  |   62 +-
 .../include/ignite/impl/ignite_environment.h    |   34 +-
 modules/platforms/cpp/core/namespaces.dox       |   74 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |    7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |   30 +
 modules/platforms/cpp/core/src/ignition.cpp     |   15 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   31 +
 .../continuous/continuous_query_handle_impl.cpp |   96 +
 .../cpp/core/src/impl/handle_registry.cpp       |  102 +-
 .../cpp/core/src/impl/ignite_environment.cpp    |  146 +-
 modules/platforms/cpp/examples/Makefile.am      |    1 +
 modules/platforms/cpp/examples/configure.ac     |    3 +-
 .../continuous-query-example/Makefile.am        |   58 +
 .../config/continuous-query-example.xml         |   52 +
 .../project/vs/continuous-query-example.vcxproj |  110 +
 .../vs/continuous-query-example.vcxproj.filters |   35 +
 .../src/continuous_query_example.cpp            |  142 +
 .../examples/include/ignite/examples/person.h   |    2 +-
 .../odbc-example/config/example-odbc.xml        |   38 +-
 .../project/vs/odbc-example.vcxproj             |    7 +-
 .../project/vs/odbc-example.vcxproj.filters     |    8 +
 .../examples/odbc-example/src/odbc_example.cpp  |  514 ++-
 .../cpp/examples/project/vs/ignite-examples.sln |    6 +
 .../putget-example/src/putget_example.cpp       |    2 +-
 .../query-example/src/query_example.cpp         |    4 +-
 .../cpp/jni/include/ignite/jni/exports.h        |    3 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   94 +-
 modules/platforms/cpp/jni/project/vs/module.def |    6 +-
 modules/platforms/cpp/jni/src/exports.cpp       |    8 -
 modules/platforms/cpp/jni/src/java.cpp          |  419 +-
 .../cpp/odbc-test/config/queries-default.xml    |  145 +
 .../odbc-test/config/queries-test-noodbc.xml    |   80 +-
 .../cpp/odbc-test/config/queries-test.xml       |   81 +-
 .../platforms/cpp/odbc-test/include/Makefile.am |    1 +
 .../cpp/odbc-test/include/complex_type.h        |  122 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |    5 +-
 .../project/vs/odbc-test.vcxproj.filters        |    3 +
 .../src/application_data_buffer_test.cpp        |    2 +-
 .../cpp/odbc-test/src/queries_test.cpp          |  513 ++-
 .../cpp/odbc-test/src/utility_test.cpp          |   27 +-
 .../platforms/cpp/odbc/include/ignite/odbc.h    |   14 +-
 .../ignite/odbc/app/application_data_buffer.h   |    4 +-
 .../odbc/include/ignite/odbc/app/parameter.h    |    2 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |    3 +
 .../cpp/odbc/include/ignite/odbc/message.h      |  138 +-
 .../odbc/include/ignite/odbc/query/data_query.h |   12 +-
 .../cpp/odbc/include/ignite/odbc/query/query.h  |   44 +-
 .../cpp/odbc/include/ignite/odbc/statement.h    |   83 +-
 .../cpp/odbc/include/ignite/odbc/type_traits.h  |    2 +-
 .../cpp/odbc/include/ignite/odbc/utility.h      |   11 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |    2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |    2 +-
 .../odbc/src/app/application_data_buffer.cpp    |   57 +-
 .../platforms/cpp/odbc/src/app/parameter.cpp    |    3 +-
 modules/platforms/cpp/odbc/src/column.cpp       |   41 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |   23 +-
 modules/platforms/cpp/odbc/src/entry_points.cpp |   32 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |   25 +-
 .../odbc/src/query/column_metadata_query.cpp    |    2 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |    6 +-
 .../cpp/odbc/src/query/foreign_keys_query.cpp   |    2 +-
 .../cpp/odbc/src/query/primary_keys_query.cpp   |    2 +-
 .../odbc/src/query/special_columns_query.cpp    |    2 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |    2 +-
 .../cpp/odbc/src/query/type_info_query.cpp      |    2 +-
 modules/platforms/cpp/odbc/src/statement.cpp    |  230 +-
 modules/platforms/cpp/odbc/src/type_traits.cpp  |    3 +
 modules/platforms/cpp/odbc/src/utility.cpp      |   24 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |    3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |    3 +
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.AspNet.nuspec                 |    4 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Binary/BinaryCompactFooterInteropTest.cs    |   31 +-
 .../Cache/Query/CacheLinqTest.cs                |   72 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   17 +-
 .../Cache/Query/CacheQueriesTest.cs             |    8 +
 .../Examples/Example.cs                         |   11 +-
 .../Examples/ExamplesTest.cs                    |  104 +-
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |   64 +-
 .../Process/IgniteProcess.cs                    |   21 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |   23 +-
 .../Services/ServicesTest.cs                    |   46 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |    3 +-
 .../Apache.Ignite.Core.csproj                   |    2 +
 .../dotnet/Apache.Ignite.Core/Binary/IBinary.cs |    3 +-
 .../Apache.Ignite.Core/Cache/Query/QueryBase.cs |   15 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |    1 +
 .../Impl/Binary/BinaryUtils.cs                  |   16 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |    4 +-
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs  |   29 +-
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |   13 +-
 .../Impl/Common/Listenable.cs                   |   49 +
 .../Impl/Compute/ComputeImpl.cs                 |    4 +-
 .../Impl/Compute/ComputeTaskHolder.cs           |   14 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |    2 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    8 -
 .../Impl/Unmanaged/UnmanagedCallbackHandlers.cs |   79 +-
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |   86 +
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        | 1229 +++---
 .../Impl/Unmanaged/UnmanagedUtils.cs            |    5 -
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../EntityFrameworkCacheTest.cs                 |   58 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/DbCommandInfo.cs                       |   21 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |   12 +-
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |    8 +
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |   34 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |    7 +
 .../Apache.Ignite/Config/ArgsConfigurator.cs    |    7 +-
 .../dotnet/Apache.Ignite/Config/Configurator.cs |   10 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |    6 +-
 modules/platforms/dotnet/DEVNOTES.txt           |   12 +-
 modules/platforms/dotnet/README.md              |  150 +
 modules/platforms/dotnet/build.bat              |   19 +
 modules/platforms/dotnet/build.ps1              |  211 +
 .../Datagrid/MultiTieredCacheExample.cs         |    8 +-
 .../Misc/ClientReconnectExample.cs              |   18 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 .../Properties/AssemblyInfo.cs                  |    6 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 .../schedule/IgniteScheduleProcessor.java       |    2 +-
 modules/schema-import-db/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/storm/pom.xml                           |    2 +-
 modules/tools/pom.xml                           |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    8 +-
 modules/twitter/pom.xml                         |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../ignite/visor/commands/VisorConsole.scala    |    1 -
 .../commands/cache/VisorCacheCommand.scala      |   37 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |  145 -
 .../commands/disco/VisorDiscoveryCommand.scala  |    2 +-
 .../commands/events/VisorEventsCommand.scala    |    8 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |   89 -
 .../testsuites/VisorConsoleSelfTestSuite.scala  |    2 -
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web-console/backend/app/agent.js        |   47 +-
 modules/web-console/backend/app/browser.js      |   26 +
 modules/web-console/backend/app/mongo.js        |   30 +-
 .../backend/config/settings.json.sample         |    7 -
 modules/web-console/backend/index.js            |    6 +-
 modules/web-console/backend/middlewares/host.js |    7 +-
 modules/web-console/backend/routes/agent.js     |    4 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 modules/web-console/backend/services/agents.js  |    2 +-
 modules/web-console/backend/services/caches.js  |    2 +
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 .../docker/compose/backend/.dockerignore        |    2 +
 .../docker/compose/backend/Dockerfile           |    6 +-
 .../web-console/docker/compose/backend/build.sh |    4 +-
 .../docker/compose/docker-compose.yml           |    5 -
 .../docker/compose/frontend/DockerfileBuild     |    4 +-
 .../docker/compose/frontend/build.sh            |    6 +-
 .../compose/frontend/nginx/web-console.conf     |    9 +
 .../web-console/docker/standalone/.dockerignore |    5 +
 .../web-console/docker/standalone/Dockerfile    |   10 +-
 modules/web-console/docker/standalone/build.sh  |    4 +-
 .../docker/standalone/docker-compose.yml        |    7 +-
 .../docker/standalone/nginx/web-console.conf    |    9 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   14 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |    2 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../frontend/app/filters/duration.filter.js     |    3 +
 .../helpers/jade/form/form-field-checkbox.jade  |    2 +-
 .../helpers/jade/form/form-field-datalist.jade  |    2 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../helpers/jade/form/form-field-password.jade  |    2 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   76 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |   11 +
 .../generator/ConfigurationGenerator.js         | 2776 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2315 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |   21 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  489 +--
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  129 -
 .../defaults/cluster.platform.provider.js       |   49 -
 .../generator/defaults/cluster.provider.js      |  293 --
 .../generator/defaults/igfs.provider.js         |   68 -
 .../configuration/generator/generator-common.js |  625 ---
 .../configuration/generator/generator-java.js   | 3617 ------------------
 .../generator/generator-optional.js             |   25 -
 .../configuration/generator/generator-spring.js | 2111 ----------
 .../app/modules/form/field/tooltip.directive.js |    2 +-
 .../app/modules/form/group/tooltip.directive.js |    2 +-
 .../app/modules/form/panel/field.directive.js   |    4 +-
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  217 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/client-near-cache.jade |    2 +-
 .../configuration/caches/concurrency.jade       |    2 +-
 .../states/configuration/caches/general.jade    |    2 +-
 .../states/configuration/caches/memory.jade     |    2 +-
 .../configuration/caches/near-cache-client.jade |    2 +-
 .../configuration/caches/near-cache-server.jade |    2 +-
 .../configuration/caches/node-filter.jade       |    4 +-
 .../states/configuration/caches/query.jade      |    5 +-
 .../states/configuration/caches/rebalance.jade  |    2 +-
 .../states/configuration/caches/statistics.jade |    2 +-
 .../states/configuration/caches/store.jade      |   20 +-
 .../states/configuration/clusters/atomic.jade   |    2 +-
 .../configuration/clusters/attributes.jade      |    2 +-
 .../states/configuration/clusters/binary.jade   |    2 +-
 .../configuration/clusters/cache-key-cfg.jade   |    2 +-
 .../configuration/clusters/checkpoint.jade      |   17 +-
 .../configuration/clusters/checkpoint/fs.jade   |   10 +-
 .../configuration/clusters/checkpoint/jdbc.jade |   47 +-
 .../configuration/clusters/checkpoint/s3.jade   |  244 +-
 .../configuration/clusters/collision.jade       |   12 +-
 .../clusters/collision/custom.jade              |    4 +-
 .../clusters/collision/fifo-queue.jade          |    2 +-
 .../clusters/collision/job-stealing.jade        |    4 +-
 .../clusters/collision/priority-queue.jade      |    2 +-
 .../configuration/clusters/communication.jade   |    2 +-
 .../configuration/clusters/connector.jade       |    2 +-
 .../configuration/clusters/deployment.jade      |  131 +-
 .../configuration/clusters/discovery.jade       |    2 +-
 .../states/configuration/clusters/events.jade   |    6 +-
 .../states/configuration/clusters/failover.jade |    6 +-
 .../states/configuration/clusters/general.jade  |    2 +-
 .../clusters/general/discovery/cloud.jade       |    2 +-
 .../clusters/general/discovery/google.jade      |    2 +-
 .../clusters/general/discovery/jdbc.jade        |    3 +-
 .../clusters/general/discovery/multicast.jade   |    2 +-
 .../clusters/general/discovery/s3.jade          |    2 +-
 .../clusters/general/discovery/shared.jade      |    2 +-
 .../clusters/general/discovery/vm.jade          |    2 +-
 .../clusters/general/discovery/zookeeper.jade   |    4 +-
 .../bounded-exponential-backoff.jade            |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    4 +-
 .../retrypolicy/exponential-backoff.jade        |    2 +-
 .../zookeeper/retrypolicy/forever.jade          |    2 +-
 .../zookeeper/retrypolicy/n-times.jade          |    2 +-
 .../zookeeper/retrypolicy/one-time.jade         |    2 +-
 .../zookeeper/retrypolicy/until-elapsed.jade    |    2 +-
 .../states/configuration/clusters/igfs.jade     |    2 +-
 .../configuration/clusters/load-balancing.jade  |   25 +-
 .../states/configuration/clusters/logger.jade   |    2 +-
 .../configuration/clusters/logger/custom.jade   |    4 +-
 .../configuration/clusters/logger/log4j.jade    |    2 +-
 .../configuration/clusters/logger/log4j2.jade   |    2 +-
 .../configuration/clusters/marshaller.jade      |    2 +-
 .../states/configuration/clusters/metrics.jade  |    2 +-
 .../states/configuration/clusters/odbc.jade     |    2 +-
 .../states/configuration/clusters/ssl.jade      |    4 +-
 .../states/configuration/clusters/swap.jade     |    2 +-
 .../states/configuration/clusters/thread.jade   |    2 +-
 .../states/configuration/clusters/time.jade     |    2 +-
 .../configuration/clusters/transactions.jade    |    2 +-
 .../states/configuration/domains/general.jade   |    2 +-
 .../states/configuration/domains/query.jade     |   16 +-
 .../states/configuration/domains/store.jade     |   12 +-
 .../modules/states/configuration/igfs/dual.jade |    2 +-
 .../states/configuration/igfs/fragmentizer.jade |    2 +-
 .../states/configuration/igfs/general.jade      |    2 +-
 .../modules/states/configuration/igfs/ipc.jade  |    2 +-
 .../modules/states/configuration/igfs/misc.jade |    6 +-
 .../states/configuration/igfs/secondary.jade    |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |   99 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   52 +-
 .../frontend/controllers/domains-controller.js  |   40 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |    4 +-
 .../frontend/gulpfile.babel.js/tasks/test.js    |   92 -
 .../gulpfile.babel.js/webpack/common.js         |   22 +-
 .../webpack/environments/development.js         |   17 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  180 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   39 +
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  124 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../frontend/views/configuration/caches.jade    |   22 +-
 .../frontend/views/configuration/clusters.jade  |   50 +-
 .../views/configuration/domains-import.jade     |    9 +-
 .../frontend/views/configuration/domains.jade   |    8 +-
 .../frontend/views/configuration/igfs.jade      |   14 +-
 .../frontend/views/configuration/summary.jade   |   27 +-
 .../frontend/views/includes/header.jade         |   15 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/settings/profile.jade        |    2 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  237 +-
 .../views/templates/agent-download.jade         |    2 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/batch-confirm.jade |    2 +-
 .../frontend/views/templates/clone.jade         |    2 +-
 .../frontend/views/templates/confirm.jade       |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 modules/web-console/pom.xml                     |    2 +-
 modules/web-console/web-agent/README.txt        |    4 +-
 .../web-agent/bin/ignite-web-agent.bat          |    7 +-
 .../web-agent/bin/ignite-web-agent.sh           |    6 +-
 modules/web-console/web-agent/pom.xml           |    2 +-
 .../console/agent/AgentConfiguration.java       |    5 +-
 .../ignite/console/agent/AgentLauncher.java     |    6 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    1 +
 modules/web/ignite-appserver-test/pom.xml       |    2 +-
 modules/web/ignite-websphere-test/pom.xml       |    2 +-
 modules/web/pom.xml                             |    2 +-
 .../internal/websession/WebSessionSelfTest.java |    4 +
 .../config/benchmark-bin-identity.properties    |   94 +
 .../config/benchmark-multicast.properties       |   15 +
 .../config/benchmark-sql-dml.properties         |   72 +
 modules/yardstick/config/ignite-base-config.xml |   71 +-
 .../config/ignite-bin-multicast-config.xml      |   86 +
 modules/yardstick/pom.xml                       |    2 +-
 .../yardstick/IgniteAbstractBenchmark.java      |   30 +
 .../yardstick/IgniteBenchmarkArguments.java     |   11 +
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |   42 +-
 .../apache/ignite/yardstick/PreloadLogger.java  |  155 +
 .../yardstick/cache/CacheEntryEventProbe.java   |    2 +-
 .../cache/IgniteBinaryIdentityBenchmark.java    |  108 +
 .../cache/IgniteBinaryIdentityGetBenchmark.java |   34 +
 .../cache/IgniteBinaryIdentityPutBenchmark.java |   35 +
 .../IgniteFieldsBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteFieldsBinaryIdentityPutBenchmark.java |   30 +
 .../yardstick/cache/IgniteIoTestBenchmark.java  |   73 +
 .../IgniteLegacyBinaryIdentityGetBenchmark.java |   30 +
 .../IgniteLegacyBinaryIdentityPutBenchmark.java |   30 +
 ...IgnitePutIfAbsentIndexedValue1Benchmark.java |   45 +
 .../IgniteReplaceIndexedValue1Benchmark.java    |   79 +
 .../cache/dml/IgniteSqlDeleteBenchmark.java     |   83 +
 .../dml/IgniteSqlDeleteFilteredBenchmark.java   |   88 +
 .../IgniteSqlInsertIndexedValue1Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue2Benchmark.java  |   48 +
 .../IgniteSqlInsertIndexedValue8Benchmark.java  |   48 +
 .../cache/dml/IgniteSqlMergeAllBenchmark.java   |   82 +
 .../cache/dml/IgniteSqlMergeBenchmark.java      |   42 +
 .../IgniteSqlMergeIndexedValue1Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue2Benchmark.java   |   43 +
 .../IgniteSqlMergeIndexedValue8Benchmark.java   |   43 +
 .../cache/dml/IgniteSqlMergeQueryBenchmark.java |  116 +
 .../cache/dml/IgniteSqlUpdateBenchmark.java     |   82 +
 .../dml/IgniteSqlUpdateFilteredBenchmark.java   |   88 +
 .../IgniteCacheRandomOperationBenchmark.java    |   22 +-
 .../yardstick/cache/model/SampleValue.java      |    2 +
 .../io/IgniteIoTestAbstractBenchmark.java       |   61 +
 .../io/IgniteIoTestSendAllBenchmark.java        |   32 +
 .../io/IgniteIoTestSendRandomBenchmark.java     |   35 +
 modules/yarn/pom.xml                            |    2 +-
 modules/zookeeper/pom.xml                       |    2 +-
 pom.xml                                         |    2 +-
 1191 files changed, 52652 insertions(+), 24120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ee73fb0/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ee73fb0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 85b9de5,7f4fb99..be22aed
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@@ -72,9 -72,10 +72,9 @@@ import org.apache.ignite.internal.binar
  import org.apache.ignite.internal.binary.BinaryMarshaller;
  import org.apache.ignite.internal.binary.GridBinaryMarshaller;
  import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage;
- import org.apache.ignite.internal.pagemem.backup.StartFullBackupAckDiscoveryMessage;
+ import org.apache.ignite.internal.pagemem.snapshot.StartFullSnapshotAckDiscoveryMessage;
  import org.apache.ignite.internal.pagemem.store.IgnitePageStoreManager;
  import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogManager;
 -import org.apache.ignite.internal.pagemem.wal.IgniteWriteAheadLogNoopManager;
  import org.apache.ignite.internal.processors.GridProcessorAdapter;
  import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ee73fb0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ee73fb0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/0ee73fb0/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[13/50] [abbrv] ignite git commit: IGNITE-4461: Hadoop: added automatic resolution of "raw" comparator for Text class.

Posted by sb...@apache.org.
IGNITE-4461: Hadoop: added automatic resolution of "raw" comparator for Text class.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: f406887c274550317e1b6fbbe1bb302f53a5eaad
Parents: beb242b
Author: devozerov <vo...@gridgain.com>
Authored: Thu Jan 5 14:48:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 5 14:48:35 2017 +0300

----------------------------------------------------------------------
 .../hadoop/impl/v2/HadoopV2TaskContext.java     | 64 ++++++++++++++------
 1 file changed, 46 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f406887c/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index e9cae1c..d328550 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -41,6 +41,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.hadoop.io.PartiallyRawComparator;
+import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
@@ -76,6 +77,8 @@ import java.io.File;
 import java.io.IOException;
 import java.security.PrivilegedExceptionAction;
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.Callable;
 
@@ -99,6 +102,9 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap
         = createHadoopLazyConcurrentMap();
 
+    /** Default partial comparator mappings. */
+    private static final Map<String, String> PARTIAL_COMPARATORS = new HashMap<>();
+
     /**
      * This method is called with reflection upon Job finish with class loader of each task.
      * This will clean up all the Fs created for specific task.
@@ -111,24 +117,6 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         fsMap.close();
     }
 
-    /**
-     * Check for combiner grouping support (available since Hadoop 2.3).
-     */
-    static {
-        boolean ok;
-
-        try {
-            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
-
-            ok = true;
-        }
-        catch (NoSuchMethodException ignore) {
-            ok = false;
-        }
-
-        COMBINE_KEY_GROUPING_SUPPORTED = ok;
-    }
-
     /** Flag is set if new context-object code is used for running the mapper. */
     private final boolean useNewMapper;
 
@@ -153,6 +141,23 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     /** Counters for task. */
     private final HadoopCounters cntrs = new HadoopCountersImpl();
 
+    static {
+        boolean ok;
+
+        try {
+            JobContext.class.getDeclaredMethod("getCombinerKeyGroupingComparator");
+
+            ok = true;
+        }
+        catch (NoSuchMethodException ignore) {
+            ok = false;
+        }
+
+        COMBINE_KEY_GROUPING_SUPPORTED = ok;
+
+        PARTIAL_COMPARATORS.put(Text.class.getName(), TextPartiallyRawComparator.class.getName());
+    }
+
     /**
      * @param taskInfo Task info.
      * @param job Job.
@@ -181,6 +186,8 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             // For map-reduce jobs prefer local writes.
             jobConf.setBooleanIfUnset(PARAM_IGFS_PREFER_LOCAL_WRITES, true);
 
+            initializePartiallyRawComparator(jobConf);
+
             jobCtx = new JobContextImpl(jobConf, new JobID(jobId.globalId().toString(), jobId.localId()));
 
             useNewMapper = jobConf.getUseNewMapper();
@@ -447,6 +454,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public Comparator<Object> groupComparator() {
         Comparator<?> res;
 
@@ -581,4 +589,24 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             throw new IgniteCheckedException(e);
         }
     }
+
+    /**
+     * Try initializing partially raw comparator for job.
+     *
+     * @param conf Configuration.
+     */
+    private void initializePartiallyRawComparator(JobConf conf) {
+        String clsName = conf.get(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), null);
+
+        if (clsName == null) {
+            Class keyCls = conf.getMapOutputKeyClass();
+
+            if (keyCls != null) {
+                clsName = PARTIAL_COMPARATORS.get(keyCls.getName());
+
+                if (clsName != null)
+                    conf.set(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), clsName);
+            }
+        }
+    }
 }
\ No newline at end of file


[06/50] [abbrv] ignite git commit: Fixed JavaDoc for RoundRobinLoadBalancingSpi

Posted by sb...@apache.org.
Fixed JavaDoc for RoundRobinLoadBalancingSpi


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 228d97bab116a513ed03f4f7f48ddeb1ef5b6577
Parents: f4a1e6c
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 29 12:21:34 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 29 12:21:34 2016 -0800

----------------------------------------------------------------------
 .../roundrobin/RoundRobinLoadBalancingSpi.java      | 16 +++++++---------
 1 file changed, 7 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/228d97ba/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
index 069641b..b0e2c78 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/loadbalancing/roundrobin/RoundRobinLoadBalancingSpi.java
@@ -51,15 +51,13 @@ import static org.apache.ignite.events.EventType.EVT_TASK_FINISHED;
 /**
  * This SPI iterates through nodes in round-robin fashion and pick the next
  * sequential node. Two modes of operation are supported: per-task and global
- * (see {@link #setPerTask(boolean)} configuration).
+ * (see {@link #setPerTask(boolean)} configuration). Global mode is used be default.
  * <p>
- * When configured in per-task mode, implementation will pick a random starting
- * node at the beginning of every task execution and then sequentially iterate through all
- * nodes in topology starting from the picked node. This is the default configuration
- * and should fit most of the use cases as it provides a fairly well-distributed
- * split and also ensures that jobs within a single task are spread out across
- * nodes to the maximum. For cases when split size is equal to the number of nodes,
- * this mode guarantees that all nodes will participate in the split.
+ * When configured in per-task mode, implementation will pick a random node at the
+ * beginning of every task execution and then sequentially iterate through all
+ * nodes in topology starting from the picked node. For cases when split size
+ * is equal to the number of nodes, this mode guarantees that all nodes will
+ * participate in the split.
  * <p>
  * When configured in global mode, a single sequential queue of nodes is maintained for
  * all tasks and the next node in the queue is picked every time. In this mode (unlike in
@@ -336,4 +334,4 @@ public class RoundRobinLoadBalancingSpi extends IgniteSpiAdapter implements Load
     @Override public String toString() {
         return S.toString(RoundRobinLoadBalancingSpi.class, this);
     }
-}
\ No newline at end of file
+}


[04/50] [abbrv] ignite git commit: Removed duplicated benchmark.

Posted by sb...@apache.org.
Removed duplicated benchmark.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 864a95e13f1262f14351df0883d0a1abd1bf70c7
Parents: 8372e69
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 14:45:08 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 14:45:08 2016 +0300

----------------------------------------------------------------------
 .../yardstick/cache/IgniteIoTestBenchmark.java  | 73 --------------------
 1 file changed, 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/864a95e1/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
deleted file mode 100644
index bee45e0..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
+++ /dev/null
@@ -1,73 +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.yardstick.cache;
-
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-import org.yardstickframework.BenchmarkConfiguration;
-import org.yardstickframework.BenchmarkUtils;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.Map;
-
-/**
- *
- */
-public class IgniteIoTestBenchmark extends IgniteAbstractBenchmark {
-    /** */
-    private List<ClusterNode> targetNodes;
-
-    /** */
-    private IgniteKernal ignite;
-
-    /** {@inheritDoc} */
-    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
-        super.setUp(cfg);
-
-        ignite = (IgniteKernal)ignite();
-
-        targetNodes = new ArrayList<>();
-
-        ClusterNode loc = ignite().cluster().localNode();
-
-        Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
-
-        for (ClusterNode node : nodes) {
-            if (!loc.equals(node))
-                targetNodes.add(node);
-        }
-
-        if (targetNodes.isEmpty())
-            throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
-
-        BenchmarkUtils.println(cfg, "Initialized target nodes: " + targetNodes + ']');
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
-
-        ignite.sendIoTest(node, null, false).get();
-
-        return true;
-    }
-}


[28/50] [abbrv] ignite git commit: IGNITE-4545 Added cache for router hostnames. - Fixes #1428.

Posted by sb...@apache.org.
IGNITE-4545 Added cache for router hostnames. - Fixes #1428.

Signed-off-by: Andrey Novikov <an...@gridgain.com>


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 27ba69b55f645a022f2e84cc4db76309b02850e3
Parents: 79401b2
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Jan 16 11:22:34 2017 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Mon Jan 16 11:22:34 2017 +0700

----------------------------------------------------------------------
 .../GridClientConnectionManagerAdapter.java     |  7 ++-
 .../impl/connection/GridClientTopology.java     | 53 ++++++++++++++++----
 2 files changed, 49 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27ba69b5/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 12baee0..f714e7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -85,6 +85,9 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
     /** Class logger. */
     private final Logger log;
 
+    /** All local enabled MACs. */
+    private final Collection<String> macs;
+
     /** NIO server. */
     private GridNioServer srv;
 
@@ -166,6 +169,8 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
         if (marshId == null && cfg.getMarshaller() == null)
             throw new GridClientException("Failed to start client (marshaller is not configured).");
 
+        macs = U.allLocalMACs();
+
         if (cfg.getProtocol() == GridClientProtocol.TCP) {
             try {
                 IgniteLogger gridLog = new JavaLogger(false);
@@ -316,7 +321,7 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
         }
 
         boolean sameHost = node.attributes().isEmpty() ||
-            F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", "));
+            F.containsAny(macs, node.attribute(ATTR_MACS).toString().split(", "));
 
         Collection<InetSocketAddress> srvs = new LinkedHashSet<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27ba69b5/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
index effd5b3..97aa586 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientTopology.java
@@ -21,7 +21,6 @@ import java.net.InetSocketAddress;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Set;
@@ -61,12 +60,18 @@ public class GridClientTopology {
     /** Cached last error prevented topology from update. */
     private GridClientException lastError;
 
+    /** Router addresses from configuration.  */
+    private final String routers;
+
     /**
      * Set of router addresses to infer direct connectivity
      * when client is working in router connection mode.
      * {@code null} when client is working in direct connection node.
      */
-    private final Set<String> routerAddrs;
+    private final Set<InetSocketAddress> routerAddrs;
+
+    /** List of all known local MACs */
+    private final Collection<String> macsCache;
 
     /** Protocol. */
     private final GridClientProtocol prot;
@@ -96,8 +101,38 @@ public class GridClientTopology {
         metricsCache = cfg.isEnableMetricsCache();
         attrCache = cfg.isEnableAttributesCache();
         prot = cfg.getProtocol();
-        routerAddrs = (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) ?
-            new HashSet<>(cfg.getRouters()) : null;
+
+        if (!cfg.getRouters().isEmpty() && cfg.getServers().isEmpty()) {
+            routers = cfg.getRouters().toString();
+
+            routerAddrs = U.newHashSet(cfg.getRouters().size());
+
+            for (String router : cfg.getRouters()) {
+                int portIdx = router.lastIndexOf(":");
+
+                if (portIdx > 0) {
+                    String hostName = router.substring(0, portIdx);
+
+                    try {
+                        int port = Integer.parseInt(router.substring(portIdx + 1));
+
+                        InetSocketAddress inetSockAddr = new InetSocketAddress(hostName, port);
+
+                        routerAddrs.add(inetSockAddr);
+                    }
+                    catch (Exception ignore) {
+                        // No-op.
+                    }
+                }
+            }
+        }
+        else {
+            routers = null;
+
+            routerAddrs = Collections.emptySet();
+        }
+
+        macsCache = U.allLocalMACs();
     }
 
     /**
@@ -279,7 +314,7 @@ public class GridClientTopology {
         try {
             if (lastError != null)
                 throw new GridClientDisconnectedException(
-                    "Topology is failed [protocol=" + prot + ", routers=" + routerAddrs + ']', lastError);
+                    "Topology is failed [protocol=" + prot + ", routers=" + routers + ']', lastError);
             else
                 return nodes.get(id);
         }
@@ -376,19 +411,17 @@ public class GridClientTopology {
             (metricsCache && attrCache) || (node.attributes().isEmpty() && node.metrics() == null);
 
         // Try to bypass object copying.
-        if (noAttrsAndMetrics && routerAddrs == null && node.connectable())
+        if (noAttrsAndMetrics && routerAddrs.isEmpty() && node.connectable())
             return node;
 
         // Return a new node instance based on the original one.
         GridClientNodeImpl.Builder nodeBuilder = GridClientNodeImpl.builder(node, !attrCache, !metricsCache);
 
         for (InetSocketAddress addr : node.availableAddresses(prot, true)) {
-            boolean router = routerAddrs == null ||
-                routerAddrs.contains(addr.getHostName() + ":" + addr.getPort()) ||
-                routerAddrs.contains(addr.getAddress().getHostAddress() + ":" + addr.getPort());
+            boolean router = routerAddrs.isEmpty() || routerAddrs.contains(addr);
 
             boolean reachable = noAttrsAndMetrics || !addr.getAddress().isLoopbackAddress() ||
-                F.containsAny(U.allLocalMACs(), node.attribute(ATTR_MACS).toString().split(", "));
+                F.containsAny(macsCache, node.<String>attribute(ATTR_MACS).split(", "));
 
             if (router && reachable) {
                 nodeBuilder.connectable(true);


[21/50] [abbrv] ignite git commit: IGNITE-3886 .NET: Fix build script to use latest build tools

Posted by sb...@apache.org.
IGNITE-3886 .NET: Fix build script to use latest build tools

This fixes C++ compilation issue with retargeted projects


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

Branch: refs/heads/ignite-gg-11810-1
Commit: ff0caf810f3bb76f284555dd37f93706c9edf1e7
Parents: 58188e8
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Tue Jan 10 17:49:07 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Tue Jan 10 17:49:07 2017 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/build.ps1 | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ff0caf81/modules/platforms/dotnet/build.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.ps1 b/modules/platforms/dotnet/build.ps1
index be7e638..4b5d937 100644
--- a/modules/platforms/dotnet/build.ps1
+++ b/modules/platforms/dotnet/build.ps1
@@ -127,7 +127,7 @@ else {
 
 # 2) Build .NET
 # Detect MSBuild 4.0+
-for ($i=4; $i -le 20; $i++) {
+for ($i=20; $i -ge 4; $i--) {
     $regKey = "HKLM:\software\Microsoft\MSBuild\ToolsVersions\$i.0"
     if (Test-Path $regKey) { break }
 }


[41/50] [abbrv] ignite git commit: Merge master into ignite-3477

Posted by sb...@apache.org.
Merge master into ignite-3477


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 4538818a36143a66dd4aaa3b9b4f819bce271545
Parents: ca373d7 7b711a3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 17 13:48:22 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 17 13:48:22 2017 +0300

----------------------------------------------------------------------
 .../utils/PlatformConfigurationUtils.java       | 128 +++++++-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |   7 -
 .../Apache.Ignite.Core.Tests.csproj             |   3 +
 .../Binary/BinaryBuilderSelfTest.cs             | 159 ++++++----
 .../BinaryBuilderSelfTestArrayIdentity.cs       |  34 +++
 .../Binary/BinaryEqualityComparerTest.cs        | 279 +++++++++++++++++
 .../Binary/IO/BinaryStreamsTest.cs              |  19 ++
 .../Cache/CacheConfigurationTest.cs             |   5 +-
 .../Cache/Query/CacheDmlQueriesTest.cs          | 296 +++++++++++++++++++
 .../IgniteConfigurationSerializerTest.cs        |  46 ++-
 .../IgniteConfigurationTest.cs                  |  28 ++
 .../Apache.Ignite.Core.csproj                   |   5 +
 .../Binary/BinaryArrayEqualityComparer.cs       | 149 ++++++++++
 .../Binary/BinaryConfiguration.cs               |  24 ++
 .../Binary/BinaryTypeConfiguration.cs           |  14 +
 .../Cache/Configuration/QueryEntity.cs          |  33 ++-
 .../Cache/Configuration/QueryField.cs           |   6 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  85 ++++--
 .../IgniteConfigurationSection.xsd              |  19 ++
 .../Apache.Ignite.Core/Impl/Binary/Binary.cs    |  28 +-
 .../Binary/BinaryEqualityComparerSerializer.cs  |  99 +++++++
 .../Impl/Binary/BinaryFieldEqualityComparer.cs  | 138 +++++++++
 .../Impl/Binary/BinaryFullTypeDescriptor.cs     |  21 +-
 .../Impl/Binary/BinaryObject.cs                 |  31 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |  62 +++-
 .../Impl/Binary/BinaryObjectHeader.cs           |  21 +-
 .../Impl/Binary/BinaryObjectSchemaHolder.cs     |  22 ++
 .../Binary/BinarySurrogateTypeDescriptor.cs     |   6 +
 .../Impl/Binary/BinarySystemHandlers.cs         |   6 +-
 .../Impl/Binary/BinaryWriter.cs                 |  11 +-
 .../Impl/Binary/DateTimeHolder.cs               |  35 ++-
 .../Impl/Binary/IBinaryEqualityComparer.cs      |  53 ++++
 .../Impl/Binary/IBinaryTypeDescriptor.cs        |   5 +
 .../Impl/Binary/Io/BinaryHeapStream.cs          |   9 +
 .../Impl/Binary/Io/BinaryStreamBase.cs          |  13 +
 .../Impl/Binary/Io/IBinaryStream.cs             |  11 +-
 .../Impl/Binary/Io/IBinaryStreamProcessor.cs    |  36 +++
 .../Impl/Binary/Marshaller.cs                   |  22 +-
 .../Impl/Binary/SerializableObjectHolder.cs     |  16 +
 .../Common/IgniteConfigurationXmlSerializer.cs  |   5 +-
 .../Impl/Memory/PlatformMemoryStream.cs         |  16 +
 41 files changed, 1845 insertions(+), 160 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4538818a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 8202e4a,c0fde97..33571ad
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@@ -17,20 -17,15 +17,26 @@@
  
  package org.apache.ignite.internal.processors.platform.utils;
  
 +import java.lang.management.ManagementFactory;
 +import java.net.InetSocketAddress;
 +import java.util.ArrayList;
 +import java.util.Collection;
++import java.util.Set;
++import java.util.HashSet;
 +import java.util.HashMap;
 +import java.util.LinkedHashMap;
 +import java.util.List;
 +import java.util.Map;
 +import javax.cache.configuration.Factory;
 +import javax.cache.expiry.ExpiryPolicy;
+ import org.apache.ignite.binary.BinaryArrayIdentityResolver;
+ import org.apache.ignite.binary.BinaryFieldIdentityResolver;
+ import org.apache.ignite.binary.BinaryIdentityResolver;
  import org.apache.ignite.binary.BinaryRawReader;
  import org.apache.ignite.binary.BinaryRawWriter;
+ import org.apache.ignite.binary.BinaryTypeConfiguration;
  import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
  import org.apache.ignite.cache.CacheAtomicityMode;
 -import org.apache.ignite.cache.CacheMemoryMode;
  import org.apache.ignite.cache.CacheMode;
  import org.apache.ignite.cache.CacheRebalanceMode;
  import org.apache.ignite.cache.CacheWriteSynchronizationMode;

http://git-wip-us.apache.org/repos/asf/ignite/blob/4538818a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index 8b8a728,5229590..c698ee3
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@@ -42,11 -42,10 +42,10 @@@ import org.apache.hadoop.util.Reflectio
  import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.hadoop.io.PartiallyRawComparator;
  import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
- import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
  import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
  import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
 -import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 -import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 +import org.apache.ignite.hadoop.HadoopInputSplit;
 +import org.apache.ignite.internal.processors.hadoop.HadoopJobEx;
  import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
  import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
  import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;


[49/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
ignite-3477 PageMemory optimizations
- use page address instead of ByteBuffer to work with page memory
- got rid of pages pin/unpin
- do not copy byte array for cache key comparison
- reduced size of data tree search row


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 7db65ddd16beae5f5bf95f67c638d5ae1c58653c
Parents: cb60e38
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jan 17 14:45:51 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jan 17 14:45:53 2017 +0300

----------------------------------------------------------------------
 .../internal/binary/BinaryEnumObjectImpl.java   |   7 +
 .../internal/binary/BinaryObjectImpl.java       |   5 +
 .../binary/BinaryObjectOffheapImpl.java         |   5 +
 .../apache/ignite/internal/pagemem/Page.java    |  11 +-
 .../ignite/internal/pagemem/PageMemory.java     |   9 +-
 .../ignite/internal/pagemem/PageUtils.java      | 185 ++++++
 .../pagemem/impl/PageMemoryNoStoreImpl.java     | 134 ++--
 .../internal/pagemem/impl/PageNoStoreImpl.java  |  91 +--
 .../delta/DataPageInsertFragmentRecord.java     |   8 +-
 .../wal/record/delta/DataPageInsertRecord.java  |   8 +-
 .../wal/record/delta/DataPageRemoveRecord.java  |   8 +-
 .../delta/DataPageSetFreeListPageRecord.java    |   8 +-
 .../wal/record/delta/FixCountRecord.java        |   8 +-
 .../record/delta/FixLeftmostChildRecord.java    |   8 +-
 .../pagemem/wal/record/delta/FixRemoveId.java   |   8 +-
 .../wal/record/delta/InitNewPageRecord.java     |   8 +-
 .../wal/record/delta/InnerReplaceRecord.java    |   4 +-
 .../pagemem/wal/record/delta/InsertRecord.java  |   5 +-
 .../pagemem/wal/record/delta/MergeRecord.java   |   3 +-
 .../wal/record/delta/MetaPageAddRootRecord.java |   8 +-
 .../wal/record/delta/MetaPageCutRootRecord.java |   7 +-
 .../wal/record/delta/MetaPageInitRecord.java    |  14 +-
 .../record/delta/MetaPageInitRootRecord.java    |   9 +-
 .../delta/MetaPageUpdateLastAllocatedIndex.java |  10 +-
 ...aPageUpdateLastSuccessfulFullSnapshotId.java |   8 +-
 .../MetaPageUpdateLastSuccessfulSnapshotId.java |   8 +-
 .../delta/MetaPageUpdateNextSnapshotId.java     |   8 +-
 .../MetaPageUpdatePartitionDataRecord.java      |  12 +-
 .../wal/record/delta/NewRootInitRecord.java     |   6 +-
 .../wal/record/delta/PageDeltaRecord.java       |   7 +-
 .../delta/PageListMetaResetCountRecord.java     |   8 +-
 .../record/delta/PagesListAddPageRecord.java    |   8 +-
 .../delta/PagesListInitNewPageRecord.java       |  10 +-
 .../record/delta/PagesListRemovePageRecord.java |   8 +-
 .../record/delta/PagesListSetNextRecord.java    |   8 +-
 .../delta/PagesListSetPreviousRecord.java       |   8 +-
 .../pagemem/wal/record/delta/RecycleRecord.java |   6 +-
 .../pagemem/wal/record/delta/RemoveRecord.java  |  12 +-
 .../pagemem/wal/record/delta/ReplaceRecord.java |   7 +-
 .../record/delta/SplitExistingPageRecord.java   |   8 +-
 .../record/delta/SplitForwardPageRecord.java    |   3 +-
 .../record/delta/TrackingPageDeltaRecord.java   |  10 +-
 .../internal/processors/cache/CacheObject.java  |   7 +
 .../processors/cache/CacheObjectAdapter.java    |  40 +-
 .../cache/CacheObjectByteArrayImpl.java         |   5 +
 .../cache/IgniteCacheOffheapManagerImpl.java    | 372 +++++++----
 .../processors/cache/database/CacheDataRow.java |  13 +-
 .../cache/database/CacheDataRowAdapter.java     |  72 ++-
 .../cache/database/CacheSearchRow.java          |  40 ++
 .../cache/database/DataStructure.java           |  36 +-
 .../IgniteCacheDatabaseSharedManager.java       |   2 +-
 .../cache/database/MetadataStorage.java         | 149 +++--
 .../cache/database/freelist/FreeListImpl.java   |  61 +-
 .../cache/database/freelist/PagesList.java      | 212 +++----
 .../database/freelist/io/PagesListMetaIO.java   |  71 +--
 .../database/freelist/io/PagesListNodeIO.java   | 117 ++--
 .../cache/database/tree/BPlusTree.java          | 616 ++++++++++---------
 .../cache/database/tree/io/BPlusIO.java         | 201 +++---
 .../cache/database/tree/io/BPlusInnerIO.java    |  71 ++-
 .../cache/database/tree/io/BPlusLeafIO.java     |  11 +-
 .../cache/database/tree/io/BPlusMetaIO.java     |  79 +--
 .../cache/database/tree/io/CacheVersionIO.java  |  46 ++
 .../cache/database/tree/io/DataPageIO.java      | 545 ++++++++--------
 .../cache/database/tree/io/DataPagePayload.java |  64 ++
 .../cache/database/tree/io/IOVersions.java      |  12 +-
 .../cache/database/tree/io/PageIO.java          | 110 +++-
 .../cache/database/tree/io/PageMetaIO.java      | 118 ++--
 .../database/tree/io/PagePartitionMetaIO.java   |  67 +-
 .../database/tree/reuse/ReuseListImpl.java      |   2 +-
 .../cache/database/tree/util/PageHandler.java   | 135 ++--
 .../database/tree/util/PageLockListener.java    |  17 +-
 .../atomic/GridDhtAtomicSingleUpdateFuture.java |   3 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   3 +
 .../apache/ignite/internal/util/GridUnsafe.java |  12 +
 .../apache/ignite/internal/util/IgniteTree.java |  31 +-
 .../internal/GridAffinityNoCacheSelfTest.java   |   5 +
 .../pagemem/impl/PageMemoryNoLoadSelfTest.java  |  29 +-
 .../IgniteIncompleteCacheObjectSelfTest.java    |   5 +
 .../database/tree/io/TrackingPageIOTest.java    |  38 +-
 .../database/BPlusTreeReuseSelfTest.java        |  22 +-
 .../processors/database/BPlusTreeSelfTest.java  |  85 +--
 .../database/FreeListImplSelfTest.java          |  15 +-
 .../database/MetadataStorageSelfTest.java       |   2 +-
 .../internal/processors/query/h2/H2Cursor.java  |   9 +-
 .../processors/query/h2/database/H2Tree.java    |   5 +-
 .../query/h2/database/H2TreeIndex.java          |   5 +-
 .../query/h2/database/io/H2InnerIO.java         |  24 +-
 .../query/h2/database/io/H2LeafIO.java          |  22 +-
 .../query/h2/database/io/H2RowLinkIO.java       |   6 +-
 .../processors/query/h2/opt/GridH2Row.java      |   5 +
 .../query/IgniteSqlDistributedJoinSelfTest.java |  46 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   5 +
 .../IgniteDistributedJoinTestSuite.java         |   2 +-
 93 files changed, 2579 insertions(+), 1807 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index a17f755..d6b6168 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@ -276,6 +276,13 @@ public class BinaryEnumObjectImpl implements BinaryObjectEx, Externalizable, Cac
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        assert valBytes != null : "Value bytes must be initialized before object is stored";
+
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), valBytes, 0);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, valBytes, 0);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
index ff14b63..0442a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectImpl.java
@@ -165,6 +165,11 @@ public final class BinaryObjectImpl extends BinaryObjectExImpl implements Extern
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        return CacheObjectAdapter.putValue(addr, cacheObjectType(), arr, start);
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, int off, int len) throws IgniteCheckedException {
         return CacheObjectAdapter.putValue(cacheObjectType(), buf, off, len, arr, start);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
index 4cf2bdf..d45815a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectOffheapImpl.java
@@ -152,6 +152,11 @@ public class BinaryObjectOffheapImpl extends BinaryObjectExImpl implements Exter
     }
 
     /** {@inheritDoc} */
+    @Override public int putValue(long addr) throws IgniteCheckedException {
+        throw new UnsupportedOperationException("TODO implement");
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean putValue(final ByteBuffer buf, final int off, final int len)
         throws IgniteCheckedException {
         throw new UnsupportedOperationException("TODO implement");

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
index e08fad6..2667e44 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/Page.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.internal.pagemem;
 
-import java.nio.ByteBuffer;
-import org.jetbrains.annotations.Nullable;
-
 /**
  *
  */
@@ -39,9 +36,9 @@ public interface Page extends AutoCloseable {
     public FullPageId fullId();
 
     /**
-     * @return ByteBuffer for modifying the page.
+     * @return Pointer for modifying the page.
      */
-    public ByteBuffer getForRead();
+    public long getForReadPointer();
 
     /**
      * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.
@@ -51,12 +48,12 @@ public interface Page extends AutoCloseable {
     /**
      * @return ByteBuffer for modifying the page.
      */
-    public ByteBuffer getForWrite();
+    public long getForWritePointer();
 
     /**
      * @return ByteBuffer for modifying the page of {@code null} if failed to get write lock.
      */
-    @Nullable public ByteBuffer tryGetForWrite();
+    public long tryGetForWritePointer();
 
     /**
      * Releases reserved page. Released page can be evicted from RAM after flushing modifications to disk.

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
index 53b37f6..cfee19f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageMemory.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.pagemem;
 
+import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.lifecycle.LifecycleAware;
@@ -25,7 +26,7 @@ import org.apache.ignite.lifecycle.LifecycleAware;
  */
 public interface PageMemory extends LifecycleAware, PageIdAllocator {
     /** */
-    ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
+    public static final ByteOrder NATIVE_BYTE_ORDER = ByteOrder.nativeOrder();
 
     /**
      * Gets the page associated with the given page ID. Each page obtained with this method must be released by
@@ -62,4 +63,10 @@ public interface PageMemory extends LifecycleAware, PageIdAllocator {
      * @return Page size with system overhead, in bytes.
      */
     public int systemPageSize();
+
+    /**
+     * @param pageAddr Page address.
+     * @return Page byte buffer.
+     */
+    public ByteBuffer pageBuffer(long pageAddr);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
new file mode 100644
index 0000000..f824368
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/PageUtils.java
@@ -0,0 +1,185 @@
+/*
+ * 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.pagemem;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/**
+ *
+ */
+@SuppressWarnings("deprecation")
+public class PageUtils {
+    /**
+     * @param addr Start address. 
+     * @param off Offset.
+     * @return Byte value from given address.
+     */
+    public static byte getByte(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getByte(addr + off);
+    }
+
+    /**
+     * @param addr Start address.
+     * @param off Offset.
+     * @param len Bytes length.
+     * @return Bytes from given address.
+     */
+    public static byte[] getBytes(long addr, int off, int len) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert len >= 0;
+
+        byte[] bytes = new byte[len];
+
+        GridUnsafe.copyMemory(null, addr + off, bytes, GridUnsafe.BYTE_ARR_OFF, len);
+
+        return bytes;
+    }
+
+    /**
+     * @param srcAddr Source address.
+     * @param srcOff Source offset.
+     * @param dst Destination array.
+     * @param dstOff Destination offset.
+     * @param len Length.
+     */
+    public static void getBytes(long srcAddr, int srcOff, byte[] dst, int dstOff, int len) {
+        assert srcAddr > 0;
+        assert srcOff > 0;
+        assert dst != null;
+        assert dstOff >= 0;
+        assert len >= 0;
+
+        GridUnsafe.copyMemory(null, srcAddr + srcOff, dst, GridUnsafe.BYTE_ARR_OFF + dstOff, len);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static short getShort(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getShort(addr + off);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static int getInt(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getInt(addr + off);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @return Value.
+     */
+    public static long getLong(long addr, int off) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        return GridUnsafe.getLong(addr + off);
+    }
+
+    /**
+     * @param addr Address/
+     * @param off Offset.
+     * @param bytes Bytes.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF, null, addr + off, bytes.length);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param bytes Bytes array.
+     * @param bytesOff Bytes array offset.
+     */
+    public static void putBytes(long addr, int off, byte[] bytes, int bytesOff) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+        assert bytes != null;
+        assert bytesOff >= 0 && (bytesOff < bytes.length || bytes.length == 0) : bytesOff;
+
+        GridUnsafe.copyMemory(bytes, GridUnsafe.BYTE_ARR_OFF + bytesOff, null, addr + off, bytes.length - bytesOff);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putByte(long addr, int off, byte v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putByte(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putShort(long addr, int off, short v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putShort(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putInt(long addr, int off, int v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putInt(addr + off, v);
+    }
+
+    /**
+     * @param addr Address.
+     * @param off Offset.
+     * @param v Value.
+     */
+    public static void putLong(long addr, int off, long v) {
+        assert addr > 0 : addr;
+        assert off >= 0;
+
+        GridUnsafe.putLong(addr + off, v);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
index 1d29df8..41e401d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageMemoryNoStoreImpl.java
@@ -20,15 +20,14 @@ package org.apache.ignite.internal.pagemem.impl;
 import java.io.Closeable;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.mem.DirectMemory;
-import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.DirectMemoryProvider;
+import org.apache.ignite.internal.mem.DirectMemoryRegion;
 import org.apache.ignite.internal.mem.OutOfMemoryException;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
@@ -37,7 +36,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.offheap.GridOffHeapOutOfMemoryException;
-import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import sun.misc.JavaNioAccess;
 import sun.misc.SharedSecrets;
@@ -90,10 +88,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     public static final int PAGE_ID_OFFSET = 8;
 
     /** Page pin counter offset. */
-    public static final int PIN_CNT_OFFSET = 16;
-
-    /** Page pin counter offset. */
-    public static final int LOCK_OFFSET = 24;
+    public static final int LOCK_OFFSET = 16;
 
     /**
      * Need a 8-byte pointer for linked list, 8 bytes for internal needs (flags),
@@ -134,21 +129,28 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     /** */
     private OffheapReadWriteLock rwLock;
 
+    /** */
+    private final boolean trackAcquiredPages;
+
     /**
+     * @param log Logger.
      * @param directMemoryProvider Memory allocator to use.
      * @param sharedCtx Cache shared context.
      * @param pageSize Page size.
+     * @param trackAcquiredPages If {@code true} tracks number of allocated pages (for tests purpose only).
      */
     public PageMemoryNoStoreImpl(
         IgniteLogger log,
         DirectMemoryProvider directMemoryProvider,
         GridCacheSharedContext<?, ?> sharedCtx,
-        int pageSize
+        int pageSize,
+        boolean trackAcquiredPages
     ) {
         assert log != null || sharedCtx != null;
 
         this.log = sharedCtx != null ? sharedCtx.logger(PageMemoryNoStoreImpl.class) : log;
         this.directMemoryProvider = directMemoryProvider;
+        this.trackAcquiredPages = trackAcquiredPages;
 
         sysPageSize = pageSize + PAGE_OVERHEAD;
 
@@ -205,6 +207,11 @@ public class PageMemoryNoStoreImpl implements PageMemory {
     }
 
     /** {@inheritDoc} */
+    @Override public ByteBuffer pageBuffer(long pageAddr) {
+        return wrapPointer(pageAddr, pageSize());
+    }
+
+    /** {@inheritDoc} */
     @Override public long allocatePage(int cacheId, int partId, byte flags) {
         long relPtr = INVALID_REL_PTR;
         long absPtr = 0;
@@ -213,13 +220,13 @@ public class PageMemoryNoStoreImpl implements PageMemory {
             relPtr = seg.borrowFreePage();
 
             if (relPtr != INVALID_REL_PTR) {
-                absPtr = seg.absolute(relPtr);
+                absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
 
                 break;
             }
         }
 
-        // No segments conatined a free page.
+        // No segments contained a free page.
         if (relPtr == INVALID_REL_PTR) {
             int segAllocIdx = nextRoundRobinIndex();
 
@@ -231,7 +238,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 relPtr = seg.allocateFreePage(flags);
 
                 if (relPtr != INVALID_REL_PTR) {
-                    absPtr = seg.absolute(relPtr);
+                    absPtr = seg.absolute(PageIdUtils.pageIndex(relPtr));
 
                     break;
                 }
@@ -248,9 +255,6 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
         writePageId(absPtr, pageId);
 
-        // Clear pin counter.
-        GridUnsafe.putLong(absPtr + PIN_CNT_OFFSET, 0);
-
         // TODO pass an argument to decide whether the page should be cleaned.
         GridUnsafe.setMemory(absPtr + PAGE_OVERHEAD, sysPageSize - PAGE_OVERHEAD, (byte)0);
 
@@ -259,7 +263,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public boolean freePage(int cacheId, long pageId) {
-        Segment seg = segment(pageId);
+        Segment seg = segment(PageIdUtils.pageIndex(pageId));
 
         seg.releaseFreePage(pageId);
 
@@ -268,25 +272,25 @@ public class PageMemoryNoStoreImpl implements PageMemory {
 
     /** {@inheritDoc} */
     @Override public Page page(int cacheId, long pageId) throws IgniteCheckedException {
-        Segment seg = segment(pageId);
+        int pageIdx = PageIdUtils.pageIndex(pageId);
 
-        return seg.acquirePage(cacheId, pageId, false);
+        Segment seg = segment(pageIdx);
+
+        return seg.acquirePage(pageIdx, pageId);
     }
 
     /** {@inheritDoc} */
     @Override public Page page(int cacheId, long pageId, boolean restore) throws IgniteCheckedException {
-        Segment seg = segment(pageId);
-
-        return seg.acquirePage(cacheId, pageId, restore);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
     @Override public void releasePage(Page p) {
-        PageNoStoreImpl page = (PageNoStoreImpl)p;
-
-        Segment seg = segments[page.segmentIndex()];
+        if (trackAcquiredPages) {
+            Segment seg = segment(PageIdUtils.pageIndex(p.id()));
 
-        seg.releasePage(page);
+            seg.onPageRelease();
+        }
     }
 
     /** {@inheritDoc} */
@@ -435,17 +439,15 @@ public class PageMemoryNoStoreImpl implements PageMemory {
      * @param absPtr Absolute memory pointer to the page header.
      * @param pageId Page ID to write.
      */
-    void writePageId(long absPtr, long pageId) {
+    private void writePageId(long absPtr, long pageId) {
         GridUnsafe.putLong(absPtr + PAGE_ID_OFFSET, pageId);
     }
 
     /**
-     * @param pageId Page ID.
+     * @param pageIdx Page index.
      * @return Segment.
      */
-    private Segment segment(long pageId) {
-        long pageIdx = PageIdUtils.pageIndex(pageId);
-
+    private Segment segment(int pageIdx) {
         int segIdx = segmentIndex(pageIdx);
 
         return segments[segIdx];
@@ -539,63 +541,31 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @return Pinned page impl.
          */
         @SuppressWarnings("TypeMayBeWeakened")
-        private PageNoStoreImpl acquirePage(int cacheId, long pageId, boolean restore) {
-            long absPtr = absolute(pageId);
-
-            long marker = GridUnsafe.getLong(absPtr);
-
-            if (marker != PAGE_MARKER)
-                throw new IllegalStateException("Page was not allocated [absPtr=" + U.hexLong(absPtr) +
-                    ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) +
-                    ", marker=" + U.hexLong(marker) + ']');
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
+        private PageNoStoreImpl acquirePage(int pageIdx, long pageId) {
+            long absPtr = absolute(pageIdx);
 
-                if (pinCnt < 0)
-                    throw new IllegalStateException("Page has been deallocated [absPtr=" + U.hexLong(absPtr) +
-                        ", cacheId=" + cacheId + ", pageId=" + U.hexLong(pageId) + ", pinCnt=" + pinCnt + ']');
+            if (trackAcquiredPages)
+                acquiredPages.incrementAndGet();
 
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt + 1))
-                    break;
-            }
-
-            acquiredPages.incrementAndGet();
-
-            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, idx, absPtr, cacheId, pageId, restore);
+            return new PageNoStoreImpl(PageMemoryNoStoreImpl.this, absPtr, pageId);
         }
 
         /**
-         * @param pinnedPage Page to unpin.
          */
-        private void releasePage(PageNoStoreImpl pinnedPage) {
-            long absPtr = pinnedPage.absolutePointer();
-
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt > 0 : "Releasing a page that was not pinned [page=" + pinnedPage +
-                    ", pinCnt=" + pinCnt + ']';
-
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, pinCnt, pinCnt - 1))
-                    break;
-            }
-
+        private void onPageRelease() {
             acquiredPages.decrementAndGet();
         }
 
         /**
-         * @param relativePtr Relative pointer.
+         * @param pageIdx Page index.
          * @return Absolute pointer.
          */
-        private long absolute(long relativePtr) {
-            int pageIdx = PageIdUtils.pageIndex(relativePtr);
-
+        private long absolute(int pageIdx) {
             pageIdx &= idxMask;
 
-            long offset = ((long)pageIdx) * sysPageSize;
+            long off = ((long)pageIdx) * sysPageSize;
 
-            return pagesBase + offset;
+            return pagesBase + off;
         }
 
         /**
@@ -616,24 +586,12 @@ public class PageMemoryNoStoreImpl implements PageMemory {
          * @param pageId Page ID to release.
          */
         private void releaseFreePage(long pageId) {
-            // Clear out flags and file ID.
-            long relPtr = PageIdUtils.pageId(0, (byte)0, PageIdUtils.pageIndex(pageId));
-
-            long absPtr = absolute(relPtr);
-
-            // Prepare page to free.
-            // First, swap pin counter down to -1.
-            while (true) {
-                long pinCnt = GridUnsafe.getLong(absPtr + PIN_CNT_OFFSET);
-
-                assert pinCnt >= 0 : "pinCnt=" + pinCnt + ", relPtr=" + U.hexLong(relPtr);
+            int pageIdx = PageIdUtils.pageIndex(pageId);
 
-                if (pinCnt > 0)
-                    throw new IllegalStateException("Releasing a page being in use: " + U.hexLong(relPtr));
+            // Clear out flags and file ID.
+            long relPtr = PageIdUtils.pageId(0, (byte)0, pageIdx);
 
-                if (GridUnsafe.compareAndSwapLong(null, absPtr + PIN_CNT_OFFSET, 0, -1))
-                    break;
-            }
+            long absPtr = absolute(pageIdx);
 
             // Second, write clean relative pointer instead of page ID.
             writePageId(absPtr, relPtr);
@@ -665,7 +623,7 @@ public class PageMemoryNoStoreImpl implements PageMemory {
                 long cnt = ((freePageRelPtrMasked & COUNTER_MASK) + COUNTER_INC) & COUNTER_MASK;
 
                 if (freePageRelPtr != INVALID_REL_PTR) {
-                    long freePageAbsPtr = absolute(freePageRelPtr);
+                    long freePageAbsPtr = absolute(PageIdUtils.pageIndex(freePageRelPtr));
 
                     long nextFreePageRelPtr = GridUnsafe.getLong(freePageAbsPtr) & ADDRESS_MASK;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
index 404c0b2..b52df55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/impl/PageNoStoreImpl.java
@@ -20,54 +20,39 @@ package org.apache.ignite.internal.pagemem.impl;
 import org.apache.ignite.internal.pagemem.FullPageId;
 import org.apache.ignite.internal.pagemem.Page;
 import org.apache.ignite.internal.pagemem.PageIdUtils;
-import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
-import org.apache.ignite.internal.util.OffheapReadWriteLock;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 
-import java.nio.ByteBuffer;
-
 /**
  *
  */
 public class PageNoStoreImpl implements Page {
     /** */
-    private int segIdx;
-
-    /** */
     private long absPtr;
 
     /** */
     private long pageId;
 
     /** */
-    private int cacheId;
-
-    /** */
     private PageMemoryNoStoreImpl pageMem;
 
-    /** */
-    private final ByteBuffer buf;
-
-    /** Page for memory restore */
-    private final boolean noTagCheck;
-
     /**
-     * @param segIdx Segment index.
+     * @param pageMem Page memory.
      * @param absPtr Absolute pointer.
+     * @param pageId Page ID.
      */
-    public PageNoStoreImpl(
-        PageMemoryNoStoreImpl pageMem, int segIdx, long absPtr, int cacheId, long pageId, boolean noTagCheck
-    ) {
+    PageNoStoreImpl(PageMemoryNoStoreImpl pageMem, long absPtr, long pageId) {
         this.pageMem = pageMem;
-        this.segIdx = segIdx;
         this.absPtr = absPtr;
 
-        this.cacheId = cacheId;
         this.pageId = pageId;
-        this.noTagCheck = noTagCheck;
+    }
 
-        buf = pageMem.wrapPointer(absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD, pageMem.pageSize());
+    /**
+     * @return Data pointer.
+     */
+    private long pointer() {
+        return absPtr + PageMemoryNoStoreImpl.PAGE_OVERHEAD;
     }
 
     /** {@inheritDoc} */
@@ -77,15 +62,15 @@ public class PageNoStoreImpl implements Page {
 
     /** {@inheritDoc} */
     @Override public FullPageId fullId() {
-        return new FullPageId(pageId, cacheId);
+        throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer getForRead() {
+    @Override public long getForReadPointer() {
         if (pageMem.readLockPage(absPtr, PageIdUtils.tag(pageId)))
-            return reset(buf.asReadOnlyBuffer());
+            return pointer();
 
-        return null;
+        return 0L;
     }
 
     /** {@inheritDoc} */
@@ -94,31 +79,29 @@ public class PageNoStoreImpl implements Page {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer getForWrite() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+    @Override public long getForWritePointer() {
+        int tag = PageIdUtils.tag(pageId);
         boolean locked = pageMem.writeLockPage(absPtr, tag);
 
-        if (!locked && !noTagCheck)
-            return null;
-
-        assert locked;
+        if (!locked)
+            return 0L;
 
-        return reset(buf);
+        return pointer();
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer tryGetForWrite() {
-        int tag =  noTagCheck ? OffheapReadWriteLock.TAG_LOCK_ALWAYS :  PageIdUtils.tag(pageId);
+    @Override public long tryGetForWritePointer() {
+        int tag = PageIdUtils.tag(pageId);
 
         if (pageMem.tryWriteLockPage(absPtr, tag))
-            return reset(buf);
+            return pointer();
 
-        return null;
+        return 0L;
     }
 
     /** {@inheritDoc} */
     @Override public void releaseWrite(boolean markDirty) {
-        long updatedPageId = PageIO.getPageId(buf);
+        long updatedPageId = PageIO.getPageId(pointer());
 
         pageMem.writeUnlockPage(absPtr, PageIdUtils.tag(updatedPageId));
     }
@@ -143,39 +126,11 @@ public class PageNoStoreImpl implements Page {
         pageMem.releasePage(this);
     }
 
-    /**
-     * @return Segment index.
-     */
-    int segmentIndex() {
-        return segIdx;
-    }
-
-    /**
-     * @return Absolute pointer to the system page start.
-     */
-    long absolutePointer() {
-        return absPtr;
-    }
-
-    /**
-     * @param buf Byte buffer.
-     * @return The given buffer back.
-     */
-    private ByteBuffer reset(ByteBuffer buf) {
-        buf.order(PageMemory.NATIVE_BYTE_ORDER);
-
-        buf.rewind();
-
-        return buf;
-    }
-
     /** {@inheritDoc} */
     @Override public String toString() {
         SB sb = new SB("PageNoStoreImpl [absPtr=0x");
 
         sb.appendHex(absPtr);
-        sb.a(", segIdx=").a(segIdx);
-        sb.a(", cacheId=").a(cacheId);
         sb.a(", pageId=0x").appendHex(pageId);
         sb.a("]");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
index 49b2626..eeaabd1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertFragmentRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 /**
@@ -50,10 +50,10 @@ public class DataPageInsertFragmentRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRowFragment(buf, payload, lastLink);
+        io.addRowFragment(pageAddr, payload, lastLink, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
index ceb06c7..f23d57a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageInsertRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 /**
@@ -51,12 +51,12 @@ public class DataPageInsertRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         assert payload != null;
 
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.addRow(buf, payload);
+        io.addRow(pageAddr, payload, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
index 511094f..17c7fe8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageRemoveRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
 
@@ -48,11 +48,11 @@ public class DataPageRemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.removeRow(buf, itemId);
+        io.removeRow(pageAddr, itemId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
index 6463989..c835052 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/DataPageSetFreeListPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.DataPageIO;
 
 import static org.apache.ignite.internal.pagemem.wal.record.WALRecord.RecordType.DATA_PAGE_SET_FREE_LIST_PAGE;
@@ -49,10 +49,10 @@ public class DataPageSetFreeListPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        DataPageIO io = DataPageIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        DataPageIO io = DataPageIO.VERSIONS.forPage(pageAddr);
 
-        io.setFreeListPageId(buf, freeListPage);
+        io.setFreeListPageId(pageAddr, freeListPage);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
index 5089c76..c727710 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixCountRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -40,10 +40,10 @@ public class FixCountRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setCount(buf, cnt);
+        io.setCount(pageAddr, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
index fe09dd1..94155c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixLeftmostChildRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -41,10 +41,10 @@ public class FixLeftmostChildRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusInnerIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusInnerIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setLeft(buf, 0, rightId);
+        io.setLeft(pageAddr, 0, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
index 5147854..b9900e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/FixRemoveId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
@@ -41,11 +41,11 @@ public class FixRemoveId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        io.setRemoveId(buf, rmvId);
+        io.setRemoveId(pageAddr, rmvId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
index 5ffc193..f5607dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InitNewPageRecord.java
@@ -17,15 +17,15 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
- * Initializes new page by calling {@link PageIO#initNewPage(ByteBuffer, long)}.
+ * Initializes new page by calling {@link PageIO#initNewPage(long, long, int)}.
  */
 public class InitNewPageRecord extends PageDeltaRecord {
     /** */
@@ -54,10 +54,10 @@ public class InitNewPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PageIO io = PageIO.getPageIO(ioType, ioVer);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(pageAddr, newPageId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
index ccc3449..35d23c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InnerReplaceRecord.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 
 /**
  * Inner replace on remove.
@@ -57,7 +57,7 @@ public class InnerReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer dstBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Inner replace record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
index f439bf3..fa598b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/InsertRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -72,8 +73,8 @@ public class InsertRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        io.insert(buf, idx, row, rowBytes, rightId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        io.insert(pageAddr, idx, row, rowBytes, rightId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
index d8113fe..84770d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MergeRecord.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -63,7 +64,7 @@ public class MergeRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer leftBuf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         throw new IgniteCheckedException("Merge record should not be logged.");
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
index 75aaca4..176df33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageAddRootRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -40,10 +40,10 @@ public class MetaPageAddRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.addRoot(buf, rootId);
+        io.addRoot(pageAddr, rootId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
index ba1d04d..50ccddc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageCutRootRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -34,10 +35,10 @@ public class MetaPageCutRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.cutRoot(buf);
+        io.cutRoot(pageAddr, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
index df671f3..3d351a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PagePartitionMetaIO;
@@ -75,15 +75,15 @@ public class MetaPageInitRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PageMetaIO io = ioType == PageIO.T_META ?
-            PageMetaIO.VERSIONS.forPage(buf) :
-            PagePartitionMetaIO.VERSIONS.forPage(buf);
+            PageMetaIO.VERSIONS.forPage(pageAddr) :
+            PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initNewPage(buf, newPageId);
+        io.initNewPage(pageAddr, newPageId, pageMem.pageSize());
 
-        io.setTreeRoot(buf, treeRoot);
-        io.setReuseListRoot(buf, reuseListRoot);
+        io.setTreeRoot(pageAddr, treeRoot);
+        io.setReuseListRoot(pageAddr, reuseListRoot);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
index a9d2677..4d56db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageInitRootRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusMetaIO;
 
 /**
@@ -40,11 +40,10 @@ public class MetaPageInitRootRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
-        throws IgniteCheckedException {
-        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.initRoot(buf, rootId);
+        io.initRoot(pageAddr, rootId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
index 836a4eb..bd9b100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastAllocatedIndex.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
@@ -39,12 +39,12 @@ public class MetaPageUpdateLastAllocatedIndex extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        assert PageIO.getType(buf) == PageIO.T_META || PageIO.getType(buf) == PageIO.T_PART_META;
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        assert PageIO.getType(pageAddr) == PageIO.T_META || PageIO.getType(pageAddr) == PageIO.T_PART_META;
 
-        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(buf));
+        PageMetaIO io = PageMetaIO.VERSIONS.forVersion(PageIO.getVersion(pageAddr));
 
-        io.setLastAllocatedIndex(buf, lastAllocatedIdx);
+        io.setLastAllocatedIndex(pageAddr, lastAllocatedIdx);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
index 5633354..e322b4f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulFullSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -38,10 +38,10 @@ public class MetaPageUpdateLastSuccessfulFullSnapshotId extends PageDeltaRecord
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulFullSnapshotId(buf, lastSuccessfulFullSnapshotId);
+        io.setLastSuccessfulFullSnapshotId(pageAddr, lastSuccessfulFullSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
index 20e136b..df9d778 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateLastSuccessfulSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -42,10 +42,10 @@ public class MetaPageUpdateLastSuccessfulSnapshotId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setLastSuccessfulSnapshotId(buf, lastSuccessfulSnapshotId);
+        io.setLastSuccessfulSnapshotId(pageAddr, lastSuccessfulSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
index 0a92aef..1403bd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdateNextSnapshotId.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageMetaIO;
 
 /**
@@ -38,10 +38,10 @@ public class MetaPageUpdateNextSnapshotId extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageMetaIO io = PageMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageMetaIO io = PageMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextSnapshotTag(buf, nextSnapshotId);
+        io.setNextSnapshotTag(pageAddr, nextSnapshotId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index ae6210a..66efc6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PagePartitionMetaIO;
 
 /**
@@ -85,12 +85,12 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagePartitionMetaIO io = PagePartitionMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.setUpdateCounter(buf, updateCntr);
-        io.setGlobalRemoveId(buf, globalRmvId);
-        io.setSize(buf, partSize);
+        io.setUpdateCounter(pageAddr, updateCntr);
+        io.setGlobalRemoveId(pageAddr, globalRmvId);
+        io.setSize(pageAddr, partSize);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
index e0284f7..26ee364 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/NewRootInitRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusInnerIO;
 
 /**
@@ -74,8 +74,8 @@ public class NewRootInitRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        io.initNewRoot(buf, newRootId, leftChildId, row, rowBytes, rightChildId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        io.initNewRoot(pageAddr, newRootId, leftChildId, row, rowBytes, rightChildId, pageMem.pageSize());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
index 9acdd08..ca52bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageDeltaRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.pagemem.wal.record.WALRecord;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -62,10 +62,11 @@ public abstract class PageDeltaRecord extends WALRecord {
      * Apply changes from this delta to the given page.
      * It is assumed that the given buffer represents page state right before this update.
      *
-     * @param buf Page buffer.
+     * @param pageMem Page memory.
+     * @param pageAddr Page address.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract void applyDelta(ByteBuffer buf) throws IgniteCheckedException;
+    public abstract void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
index 23ebcee..16587d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PageListMetaResetCountRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListMetaIO;
 
 /**
@@ -34,10 +34,10 @@ public class PageListMetaResetCountRecord extends PageDeltaRecord  {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListMetaIO io = PagesListMetaIO.VERSIONS.forPage(pageAddr);
 
-        io.resetCount(buf);
+        io.resetCount(pageAddr);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
index 7ea16b8..a503b46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListAddPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -51,10 +51,10 @@ public class PagesListAddPageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        int cnt = io.addPage(buf, dataPageId);
+        int cnt = io.addPage(pageAddr, dataPageId, pageMem.pageSize());
 
         assert cnt >= 0 : cnt;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
index 922f212..b035323 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListInitNewPageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -73,14 +73,14 @@ public class PagesListInitNewPageRecord extends InitNewPageRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
         PagesListNodeIO io = PageIO.getPageIO(PageIO.T_PAGE_LIST_NODE, ioVer);
 
-        io.initNewPage(buf, pageId());
-        io.setPreviousId(buf, prevPageId);
+        io.initNewPage(pageAddr, pageId(), pageMem.pageSize());
+        io.setPreviousId(pageAddr, prevPageId);
 
         if (addDataPageId != 0L) {
-            int cnt = io.addPage(buf, addDataPageId);
+            int cnt = io.addPage(pageAddr, addDataPageId, pageMem.pageSize());
 
             assert cnt == 0 : cnt;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
index 128e396..26f832c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListRemovePageRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -51,10 +51,10 @@ public class PagesListRemovePageRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        boolean rmvd = io.removePage(buf, rmvdPageId);
+        boolean rmvd = io.removePage(pageAddr, rmvdPageId);
 
         assert rmvd;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
index 07ce6d2..c0bed60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetNextRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
 /**
@@ -47,10 +47,10 @@ public class PagesListSetNextRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setNextId(buf, nextPageId);
+        io.setNextId(pageAddr, nextPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
index bb3a877..21c3ef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/PagesListSetPreviousRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.freelist.io.PagesListNodeIO;
 
 /**
@@ -47,10 +47,10 @@ public class PagesListSetPreviousRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PagesListNodeIO io = PagesListNodeIO.VERSIONS.forPage(pageAddr);
 
-        io.setPreviousId(buf, prevPageId);
+        io.setPreviousId(pageAddr, prevPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
index 470f441..1737e12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RecycleRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 
 /**
@@ -40,8 +40,8 @@ public class RecycleRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        PageIO.setPageId(buf, newPageId);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        PageIO.setPageId(pageAddr, newPageId);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
index 453525c..6d278e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/RemoveRecord.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.pagemem.wal.record.delta;
 
-import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 import org.apache.ignite.internal.processors.cache.database.tree.io.PageIO;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -47,13 +47,13 @@ public class RemoveRecord extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf) throws IgniteCheckedException {
-        BPlusIO<?> io = PageIO.getBPlusIO(buf);
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr) throws IgniteCheckedException {
+        BPlusIO<?> io = PageIO.getBPlusIO(pageAddr);
 
-        if (io.getCount(buf) != cnt)
-            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(buf) + ']');
+        if (io.getCount(pageAddr) != cnt)
+            throw new DeltaApplicationException("Count is wrong [expCnt=" + cnt + ", actual=" + io.getCount(pageAddr) + ']');
 
-        io.remove(buf, idx, cnt);
+        io.remove(pageAddr, idx, cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
index 8a9e4e3..757afe1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/ReplaceRecord.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.pagemem.wal.record.delta;
 
 import java.nio.ByteBuffer;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.pagemem.PageMemory;
 import org.apache.ignite.internal.processors.cache.database.tree.io.BPlusIO;
 
 /**
@@ -55,12 +56,12 @@ public class ReplaceRecord<L> extends PageDeltaRecord {
     }
 
     /** {@inheritDoc} */
-    @Override public void applyDelta(ByteBuffer buf)
+    @Override public void applyDelta(PageMemory pageMem, long pageAddr)
         throws IgniteCheckedException {
-        if (io.getCount(buf) < idx)
+        if (io.getCount(pageAddr) < idx)
             throw new DeltaApplicationException("Index is greater than count: " + idx);
 
-        io.store(buf, idx, row, rowBytes);
+        io.store(pageAddr, idx, row, rowBytes);
     }
 
     /** {@inheritDoc} */


[23/50] [abbrv] ignite git commit: IGNITE-4531: SQL: Use correct property name in BinaryProperty. This closes 1419.

Posted by sb...@apache.org.
IGNITE-4531: SQL: Use correct property name in BinaryProperty. This closes 1419.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: abc8b9091e9eda8cc43f398939bf1fb6035f7a3f
Parents: 1f358db
Author: Alexander Paschenko <al...@gmail.com>
Authored: Thu Jan 12 10:50:14 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 12 10:50:14 2017 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java      |  2 +-
 ...IgniteCacheAbstractInsertSqlQuerySelfTest.java | 14 +++++++-------
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java   |  6 +++---
 .../cache/IgniteCacheInsertSqlQuerySelfTest.java  | 18 ++++++++++--------
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java   | 14 +++++++-------
 .../cache/IgniteCacheUpdateSqlQuerySelfTest.java  |  4 ++--
 6 files changed, 30 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/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 58f94f4..3286bac 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
@@ -2147,7 +2147,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (!(obj instanceof BinaryObjectBuilder))
                 throw new UnsupportedOperationException("Individual properties can be set for binary builders only");
 
-            setValue0((BinaryObjectBuilder) obj, name(), propVal, type());
+            setValue0((BinaryObjectBuilder) obj, propName, propVal, type());
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
index df4259e..86d01c7 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractInsertSqlQuerySelfTest.java
@@ -151,7 +151,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             s2p.setFields(flds);
 
@@ -172,7 +172,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             i2p.setFields(flds);
 
@@ -194,7 +194,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             flds.put("key", Integer.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k2p.setFields(flds);
 
@@ -216,7 +216,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
 
             flds.put("Id", Integer.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
             flds.put("IntVal", Integer.class.getName());
 
             k22p.setFields(flds);
@@ -240,7 +240,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             flds.put("key", Integer.class.getName());
             flds.put("strKey", String.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k32p.setFields(flds);
 
@@ -263,7 +263,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
             flds.put("key", Integer.class.getName());
             flds.put("strKey", String.class.getName());
             flds.put("id", Integer.class.getName());
-            flds.put("name", String.class.getName());
+            flds.put("firstName", String.class.getName());
 
             k42p.setFields(flds);
 
@@ -515,7 +515,7 @@ public abstract class IgniteCacheAbstractInsertSqlQuerySelfTest extends GridComm
         protected int id;
 
         /** */
-        @QuerySqlField
+        @QuerySqlField(name = "firstName")
         protected String name;
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
index 7f79ec4..649012f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractSqlDmlQuerySelfTest.java
@@ -139,7 +139,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         BinaryObjectBuilder bldr = ignite(0).binary().builder("Person");
 
         bldr.setField("id", id);
-        bldr.setField("name", name);
+        bldr.setField("firstName", name);
         bldr.setField("secondName", secondName);
 
         return bldr.build();
@@ -186,7 +186,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         LinkedHashMap<String, String> flds = new LinkedHashMap<>();
 
         flds.put("id", Integer.class.getName());
-        flds.put("name", String.class.getName());
+        flds.put("firstName", String.class.getName());
         flds.put("secondName", String.class.getName());
 
         e.setFields(flds);
@@ -214,7 +214,7 @@ public abstract class IgniteCacheAbstractSqlDmlQuerySelfTest extends GridCommonA
         protected int id;
 
         /** */
-        @QuerySqlField
+        @QuerySqlField(name = "firstName")
         protected final String name;
 
         /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
index 04a352f..e9c21dc 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInsertSqlQuerySelfTest.java
@@ -56,7 +56,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     public void testInsertWithExplicitKey() {
         IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("insert into Person (_key, id, name) values ('s', ?, ?), " +
+        p.query(new SqlFieldsQuery("insert into Person (_key, id, firstName) values ('s', ?, ?), " +
             "('a', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get("s"));
@@ -76,7 +76,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         assertEquals("Sergi", p.get("s"));
         assertEquals("Alex", p.get("a"));
 
-        p.query(new SqlFieldsQuery("insert into Person(_key, id, name) " +
+        p.query(new SqlFieldsQuery("insert into Person(_key, id, firstName) " +
             "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
 
         assertEquals(createPerson(5, "Sergi"), p.get("se"));
@@ -91,7 +91,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (_key, id, name) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
+            "insert into Person (_key, id, firstName) values (cast('1' as int), ?, ?), (2, (5 - 3), 'Alex')")
             .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(1));
@@ -106,7 +106,7 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, id, firstName) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
 
@@ -119,8 +119,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
-            "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
+        p.query(new SqlFieldsQuery("insert into \"Person2\" (\"Id\", \"id\", \"firstName\", \"IntVal\") " +
+            "values (1, ?, ?, 5),  (2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
 
@@ -177,7 +177,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')")
+            .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
 
@@ -194,7 +195,8 @@ public class IgniteCacheInsertSqlQuerySelfTest extends IgniteCacheAbstractInsert
         IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "insert into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "insert into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')")
+            .setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
index 0ff3fda..58d07af 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheMergeSqlQuerySelfTest.java
@@ -32,7 +32,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
     public void testMergeWithExplicitKey() {
         IgniteCache<String, Person> p = ignite(0).cache("S2P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("merge into Person (_key, id, name) values ('s', ?, ?), " +
+        p.query(new SqlFieldsQuery("merge into Person (_key, id, firstName) values ('s', ?, ?), " +
             "('a', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get("s"));
@@ -52,7 +52,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         assertEquals("Sergi", p.get("s"));
         assertEquals("Alex", p.get("a"));
 
-        p.query(new SqlFieldsQuery("merge into Person(_key, id, name) " +
+        p.query(new SqlFieldsQuery("merge into Person(_key, id, firstName) " +
             "(select substring(lower(_val), 0, 2), cast(length(_val) as int), _val from String)"));
 
         assertEquals(createPerson(5, "Sergi"), p.get("se"));
@@ -67,7 +67,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Integer, Person> p = ignite(0).cache("I2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (_key, id, name) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
+            "merge into Person (_key, id, firstName) values (cast(? as int), ?, ?), (2, (5 - 3), 'Alex')")
             .setArgs("1", 1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(1));
@@ -82,7 +82,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key, Person> p = ignite(0).cache("K2P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, id, name) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, id, firstName) values (1, ?, ?), (2, 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key(1)));
 
@@ -95,7 +95,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
     public void testFieldsCaseSensitivity() {
         IgniteCache<Key2, Person> p = ignite(0).cache("K22P").withKeepBinary();
 
-        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"name\", \"IntVal\") values (1, ?, ?, 5), " +
+        p.query(new SqlFieldsQuery("merge into \"Person2\" (\"Id\", \"id\", \"firstName\", \"IntVal\") values (1, ?, ?, 5), " +
             "(2, 3, 'Alex', 6)").setArgs(4, "Sergi"));
 
         assertEquals(createPerson2(4, "Sergi", 5), p.get(new Key2(1)));
@@ -127,7 +127,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key3, Person> p = ignite(0).cache("K32P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key3(1)));
 
@@ -144,7 +144,7 @@ public class IgniteCacheMergeSqlQuerySelfTest extends IgniteCacheAbstractInsertS
         IgniteCache<Key4, Person> p = ignite(0).cache("K42P").withKeepBinary();
 
         p.query(new SqlFieldsQuery(
-            "merge into Person (key, strKey, id, name) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
+            "merge into Person (key, strKey, id, firstName) values (1, 'aa', ?, ?), (2, 'bb', 2, 'Alex')").setArgs(1, "Sergi"));
 
         assertEquals(createPerson(1, "Sergi"), p.get(new Key4(1)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/abc8b909/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
index 332a082..58bcaac 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheUpdateSqlQuerySelfTest.java
@@ -61,8 +61,8 @@ public class IgniteCacheUpdateSqlQuerySelfTest extends IgniteCacheAbstractSqlDml
     public void testUpdateSimple() {
         IgniteCache p = cache();
 
-        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.name = " +
-            "substring(p.name, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
+        QueryCursor<List<?>> c = p.query(new SqlFieldsQuery("update Person p set p.id = p.id * 2, p.firstName = " +
+            "substring(p.firstName, 0, 2) where length(p._key) = ? or p.secondName like ?").setArgs(2, "%ite"));
 
         c.iterator();
 


[33/50] [abbrv] ignite git commit: IGNITE-4514: Hadoop: fixed intermittent intermittent failure in HadoopCommandLineTest.testHiveCommandLine. This closes #1397.

Posted by sb...@apache.org.
IGNITE-4514: Hadoop: fixed intermittent intermittent failure in HadoopCommandLineTest.testHiveCommandLine. This closes #1397.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: d6d42c2a17b7cf53a71d59931b2cf838bd8116c9
Parents: 82dd912
Author: Ivan Veselovskiy <iv...@gridgain.com>
Authored: Mon Jan 16 16:46:20 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Jan 16 16:46:20 2017 +0300

----------------------------------------------------------------------
 .../processors/hadoop/impl/v2/HadoopV2TaskContext.java        | 7 -------
 1 file changed, 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d6d42c2a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index d328550..5229590 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -42,7 +42,6 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.hadoop.io.PartiallyRawComparator;
 import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
-import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
@@ -508,12 +507,6 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         FileSystem fs;
 
         try {
-            // This assertion uses .startsWith() instead of .equals() because task class loaders may
-            // be reused between tasks of the same job.
-            assert ((HadoopClassLoader)getClass().getClassLoader()).name()
-                .startsWith(HadoopClassLoader.nameForTask(taskInfo(), true));
-
-            // We also cache Fs there, all them will be cleared explicitly upon the Job end.
             fs = fileSystemForMrUserWithCaching(jobDir.toUri(), jobConf(), fsMap);
         }
         catch (IOException e) {


[15/50] [abbrv] ignite git commit: IGNITE-4412 fix NLogLoggerTest thread safety issue

Posted by sb...@apache.org.
IGNITE-4412 fix NLogLoggerTest thread safety issue

This closes #1401


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

Branch: refs/heads/ignite-gg-11810-1
Commit: bf118aad8d8251144062d97c476fbe5f817d8018
Parents: d2e6007
Author: Sergey Stronchinskiy <se...@kraftvaerk.com>
Authored: Mon Jan 9 15:36:11 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Mon Jan 9 15:36:11 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests.csproj             |  1 +
 .../Log/ConcurrentMemoryTarget.cs               | 73 ++++++++++++++++++++
 .../Log/NLogLoggerTest.cs                       |  5 +-
 3 files changed, 76 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 5948593..55adfe4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -77,6 +77,7 @@
     <Compile Include="Collections\ReadOnlyCollectionTest.cs" />
     <Compile Include="Collections\ReadOnlyDictionaryTest.cs" />
     <Compile Include="Common\IgniteGuidTest.cs" />
+    <Compile Include="Log\ConcurrentMemoryTarget.cs" />
     <Compile Include="Log\DefaultLoggerTest.cs" />
     <Compile Include="Log\Log4NetLoggerTest.cs" />
     <Compile Include="Log\NLogLoggerTest.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
new file mode 100644
index 0000000..66bdbe2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/ConcurrentMemoryTarget.cs
@@ -0,0 +1,73 @@
+\ufeff/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Log
+{
+    using System.Collections.Generic;
+    using System.Linq;
+    using global::NLog;
+    using global::NLog.Targets;
+
+    /// <summary>
+    /// NLog target which supports logging from multiple threads.
+    /// </summary>
+    public class ConcurrentMemoryTarget : TargetWithLayout
+    {
+        /// <summary> Object used for locking. </summary>
+        private readonly object _locker = new object();
+
+        /// <summary> Logs. </summary>
+        private readonly IList<string> _logs;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="ConcurrentMemoryTarget" /> class.
+        /// </summary>
+        public ConcurrentMemoryTarget()
+        {
+            _logs = new List<string>();
+            Name = "ConcurrentMemoryTarget";
+        }
+
+        /// <summary>
+        /// Gets the collection of logs gathered in the <see cref="ConcurrentMemoryTarget" />.
+        /// </summary>
+        public IEnumerable<string> Logs
+        {
+            get
+            {
+                lock (_locker)
+                {
+                    return _logs.ToList();
+                }
+            }
+        }
+
+        /// <summary>
+        /// Renders the logging event message and adds it to the internal ArrayList of log messages.
+        /// </summary>
+        /// <param name="logEvent">The logging event.</param>
+        protected override void Write(LogEventInfo logEvent)
+        {
+            lock (_locker)
+            {
+                var msg = Layout.Render(logEvent);
+
+                _logs.Add(msg);
+            }
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf118aad/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
index 7806ecd..2743353 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Log/NLogLoggerTest.cs
@@ -25,7 +25,6 @@ namespace Apache.Ignite.Core.Tests.Log
     using global::NLog;
     using global::NLog.Config;
     using global::NLog.Layouts;
-    using global::NLog.Targets;
     using NUnit.Framework;
     using LogLevel = Apache.Ignite.Core.Log.LogLevel;
 
@@ -35,7 +34,7 @@ namespace Apache.Ignite.Core.Tests.Log
     public class NLogLoggerTest
     {
         /** */
-        private MemoryTarget _logTarget;
+        private ConcurrentMemoryTarget _logTarget;
 
         /// <summary>
         /// Test set up.
@@ -45,7 +44,7 @@ namespace Apache.Ignite.Core.Tests.Log
         {
             var cfg = new LoggingConfiguration();
 
-            _logTarget = new MemoryTarget("mem")
+            _logTarget = new ConcurrentMemoryTarget
             {
                 Layout = new SimpleLayout("${Logger}|${Level}|${Message}|${exception}|${all-event-properties}")
             };


[05/50] [abbrv] ignite git commit: For communication spi disabled pairedConnections by default, implemented NIO sessions balancing for this mode.

Posted by sb...@apache.org.
For communication spi disabled pairedConnections by default, implemented NIO sessions balancing for this mode.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: da5b68cc89ba6eeff25beb66e3a4d8c2b9871ab3
Parents: 864a95e
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 29 15:46:59 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 29 15:46:59 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioServer.java | 159 ++++++++++++++++---
 .../communication/tcp/TcpCommunicationSpi.java  |  20 +--
 .../tcp/TcpCommunicationSpiMBean.java           |   5 +-
 ...mmunicationBalancePairedConnectionsTest.java |  28 ++++
 .../IgniteCommunicationBalanceTest.java         |  25 ++-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 ------
 ...micMessageRecoveryPairedConnectionsTest.java |  47 ++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 8 files changed, 250 insertions(+), 87 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/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 bc1f173..a59adba 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
@@ -227,6 +227,9 @@ public class GridNioServer<T> {
     /** */
     private final IgniteRunnable balancer;
 
+    /** */
+    private final boolean readWriteSelectorsAssign;
+
     /**
      * @param addr Address.
      * @param port Port.
@@ -250,7 +253,7 @@ public class GridNioServer<T> {
      * @param writerFactory Writer factory.
      * @param skipRecoveryPred Skip recovery predicate.
      * @param msgQueueLsnr Message queue size listener.
-     * @param balancing NIO sessions balancing flag.
+     * @param readWriteSelectorsAssign If {@code true} then in/out connections are assigned to even/odd workers.
      * @param filters Filters for this server.
      * @throws IgniteCheckedException If failed.
      */
@@ -275,7 +278,7 @@ public class GridNioServer<T> {
         GridNioMessageWriterFactory writerFactory,
         IgnitePredicate<Message> skipRecoveryPred,
         IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
-        boolean balancing,
+        boolean readWriteSelectorsAssign,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         if (port != -1)
@@ -300,6 +303,7 @@ public class GridNioServer<T> {
         this.sndQueueLimit = sndQueueLimit;
         this.msgQueueLsnr = msgQueueLsnr;
         this.selectorSpins = selectorSpins;
+        this.readWriteSelectorsAssign = readWriteSelectorsAssign;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -359,10 +363,16 @@ public class GridNioServer<T> {
 
         IgniteRunnable balancer0 = null;
 
-        if (balancing && balancePeriod > 0) {
+        if (balancePeriod > 0) {
             boolean rndBalance = IgniteSystemProperties.getBoolean(IGNITE_IO_BALANCE_RANDOM_BALANCE, false);
 
-            balancer0 = rndBalance ? new RandomBalancer() : new SizeBasedBalancer(balancePeriod);
+            if (rndBalance)
+                balancer0 = new RandomBalancer();
+            else {
+                balancer0 = readWriteSelectorsAssign ?
+                    new ReadWriteSizeBasedBalancer(balancePeriod) :
+                    new SizeBasedBalancer(balancePeriod);
+            }
         }
 
         this.balancer = balancer0;
@@ -823,21 +833,31 @@ public class GridNioServer<T> {
         int balanceIdx;
 
         if (workers > 1) {
-            if (req.accepted()) {
-                balanceIdx = readBalanceIdx;
+            if (readWriteSelectorsAssign) {
+                if (req.accepted()) {
+                    balanceIdx = readBalanceIdx;
 
-                readBalanceIdx += 2;
+                    readBalanceIdx += 2;
 
-                if (readBalanceIdx >= workers)
-                    readBalanceIdx = 0;
+                    if (readBalanceIdx >= workers)
+                        readBalanceIdx = 0;
+                }
+                else {
+                    balanceIdx = writeBalanceIdx;
+
+                    writeBalanceIdx += 2;
+
+                    if (writeBalanceIdx >= workers)
+                        writeBalanceIdx = 1;
+                }
             }
             else {
-                balanceIdx = writeBalanceIdx;
+                balanceIdx = readBalanceIdx;
 
-                writeBalanceIdx += 2;
+                readBalanceIdx++;
 
-                if (writeBalanceIdx >= workers)
-                    writeBalanceIdx = 1;
+                if (readBalanceIdx >= workers)
+                    readBalanceIdx = 0;
             }
         }
         else
@@ -3124,8 +3144,8 @@ public class GridNioServer<T> {
         /** */
         private long selectorSpins;
 
-        /** NIO sessions balancing flag. */
-        private boolean balancing;
+        /** */
+        private boolean readWriteSelectorsAssign;
 
         /**
          * Finishes building the instance.
@@ -3155,7 +3175,7 @@ public class GridNioServer<T> {
                 writerFactory,
                 skipRecoveryPred,
                 msgQueueLsnr,
-                balancing,
+                readWriteSelectorsAssign,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -3169,11 +3189,11 @@ public class GridNioServer<T> {
         }
 
         /**
-         * @param balancing NIO sessions balancing flag.
+         * @param readWriteSelectorsAssign {@code True} to assign in/out connections even/odd workers.
          * @return This for chaining.
          */
-        public Builder<T> balancing(boolean balancing) {
-            this.balancing = balancing;
+        public Builder<T> readWriteSelectorsAssign(boolean readWriteSelectorsAssign) {
+            this.readWriteSelectorsAssign = readWriteSelectorsAssign;
 
             return this;
         }
@@ -3415,7 +3435,7 @@ public class GridNioServer<T> {
     /**
      *
      */
-    private class SizeBasedBalancer implements IgniteRunnable {
+    private class ReadWriteSizeBasedBalancer implements IgniteRunnable {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -3428,7 +3448,7 @@ public class GridNioServer<T> {
         /**
          * @param balancePeriod Period.
          */
-        SizeBasedBalancer(long balancePeriod) {
+        ReadWriteSizeBasedBalancer(long balancePeriod) {
             this.balancePeriod = balancePeriod;
         }
 
@@ -3559,6 +3579,100 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    private class SizeBasedBalancer implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private long lastBalance;
+
+        /** */
+        private final long balancePeriod;
+
+        /**
+         * @param balancePeriod Period.
+         */
+        SizeBasedBalancer(long balancePeriod) {
+            this.balancePeriod = balancePeriod;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            long now = U.currentTimeMillis();
+
+            if (lastBalance + balancePeriod < now) {
+                lastBalance = now;
+
+                long maxBytes0 = -1, minBytes0 = -1;
+                int maxBytesIdx = -1, minBytesIdx = -1;
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    int sesCnt = worker.workerSessions.size();
+
+                    long bytes0 = worker.bytesRcvd0 + worker.bytesSent0;
+
+                    if ((maxBytes0 == -1 || bytes0 > maxBytes0) && bytes0 > 0 && sesCnt > 1) {
+                        maxBytes0 = bytes0;
+                        maxBytesIdx = i;
+                    }
+
+                    if (minBytes0 == -1 || bytes0 < minBytes0) {
+                        minBytes0 = bytes0;
+                        minBytesIdx = i;
+                    }
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Balancing data [min0=" + minBytes0 + ", minIdx=" + minBytesIdx +
+                        ", max0=" + maxBytes0 + ", maxIdx=" + maxBytesIdx + ']');
+
+                if (maxBytes0 != -1 && minBytes0 != -1) {
+                    GridSelectorNioSessionImpl ses = null;
+
+                    long bytesDiff = maxBytes0 - minBytes0;
+                    long delta = bytesDiff;
+                    double threshold = bytesDiff * 0.9;
+
+                    GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+                        clientWorkers.get(maxBytesIdx).workerSessions;
+
+                    for (GridSelectorNioSessionImpl ses0 : sessions) {
+                        long bytesSent0 = ses0.bytesSent0();
+
+                        if (bytesSent0 < threshold &&
+                            (ses == null || delta > U.safeAbs(bytesSent0 - bytesDiff / 2))) {
+                            ses = ses0;
+                            delta = U.safeAbs(bytesSent0 - bytesDiff / 2);
+                        }
+                    }
+
+                    if (ses != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Will move session to less loaded worker [ses=" + ses +
+                                ", from=" + maxBytesIdx + ", to=" + minBytesIdx + ']');
+
+                        moveSession(ses, maxBytesIdx, minBytesIdx);
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to find session to move.");
+                    }
+                }
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    worker.reset0();
+                }
+            }
+        }
+    }
+
+    /**
      * For tests only.
      */
     @SuppressWarnings("unchecked")
@@ -3625,6 +3739,9 @@ public class GridNioServer<T> {
      *
      */
     interface SessionChangeRequest {
+        /**
+         * @return Session.
+         */
         GridNioSession session();
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/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 c35b5ef..ae0e6f0 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
@@ -293,7 +293,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /**
      * Default count of selectors for TCP server equals to
-     * {@code "Math.min(8, Runtime.getRuntime().availableProcessors())"}.
+     * {@code "Math.max(4, Runtime.getRuntime().availableProcessors() / 2)"}.
      */
     public static final int DFLT_SELECTORS_CNT = Math.max(4, Runtime.getRuntime().availableProcessors() / 2);
 
@@ -979,7 +979,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     private IpcSharedMemoryServerEndpoint shmemSrv;
 
     /** */
-    private boolean usePairedConnections = true;
+    private boolean usePairedConnections;
 
     /** */
     private int connectionsPerNode = DFLT_CONN_PER_NODE;
@@ -1193,10 +1193,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Set this to {@code false} if each connection of {@link #getConnectionsPerNode()}
      * should be used for outgoing and incoming messages. In this case total number
      * of connections between local and each remote node is {@link #getConnectionsPerNode()}.
-     * In this case load NIO selectors load
-     * balancing of {@link GridNioServer} will be disabled.
      * <p>
-     * Default is {@code true}.
+     * Default is {@code false}.
      *
      * @param usePairedConnections {@code true} to use paired connections and {@code false} otherwise.
      * @see #getConnectionsPerNode()
@@ -2057,16 +2055,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .writerFactory(writerFactory)
                         .skipRecoveryPredicate(skipRecoveryPred)
                         .messageQueueSizeListener(queueSizeMonitor)
-                        .balancing(usePairedConnections) // Current balancing logic assumes separate in/out connections.
+                        .readWriteSelectorsAssign(usePairedConnections)
                         .build();
 
                 boundTcpPort = port;
 
                 // Ack Port the TCP server was bound to.
-                if (log.isInfoEnabled())
+                if (log.isInfoEnabled()) {
                     log.info("Successfully bound communication NIO server to TCP port " +
-                        "[port=" + boundTcpPort + ", locHost=" + locHost + ", selectorsCnt=" + selectorsCnt +
-                        ", selectorSpins=" + srvr.selectorSpins() + ']');
+                        "[port=" + boundTcpPort +
+                        ", locHost=" + locHost +
+                        ", selectorsCnt=" + selectorsCnt +
+                        ", selectorSpins=" + srvr.selectorSpins() +
+                        ", pairedConn=" + usePairedConnections + ']');
+                }
 
                 srvr.idleTimeout(idleConnTimeout);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/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 c7a1a53..c56e18c 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
@@ -51,10 +51,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * is {@link #getConnectionsPerNode()} * 2.
      * <p>
      * Returns {@code false} if each connection of {@link #getConnectionsPerNode()}
-     * should be used for outgoing and incoming messages. In this case load NIO selectors load
-     * balancing of {@link GridNioServer} will be disabled.
+     * should be used for outgoing and incoming messages.
      * <p>
-     * Default is {@code true}.
+     * Default is {@code false}.
      *
      * @return {@code true} to use paired connections and {@code false} otherwise.
      * @see #getConnectionsPerNode()

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalancePairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalancePairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalancePairedConnectionsTest.java
new file mode 100644
index 0000000..4544030
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalancePairedConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.managers.communication;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalancePairedConnectionsTest extends IgniteCommunicationBalanceTest {
+    /** {@inheritDoc} */
+    @Override protected boolean usePairedConnections() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
index e142aef..4271417 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
@@ -63,6 +63,7 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
 
         commSpi.setSharedMemoryPort(-1);
         commSpi.setConnectionsPerNode(connectionsPerNode());
+        commSpi.setUsePairedConnections(usePairedConnections());
 
         if (selectors > 0)
             commSpi.setSelectorsCount(selectors);
@@ -75,6 +76,13 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Value for {@link TcpCommunicationSpi#setUsePairedConnections(boolean)}.
+     */
+    protected boolean usePairedConnections() {
+        return false;
+    }
+
+    /**
      * @return Connections per node.
      */
     protected int connectionsPerNode() {
@@ -97,7 +105,7 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
         try {
             selectors = 4;
 
-            final int SRVS = 4;
+            final int SRVS = 6;
 
             startGridsMultiThreaded(SRVS);
 
@@ -105,7 +113,7 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
 
             final Ignite client = startGrid(SRVS);
 
-            for (int i = 0; i < 4; i++) {
+            for (int i = 0; i < SRVS; i++) {
                 ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
 
                 client.compute(client.cluster().forNode(node)).call(new DummyCallable(null));
@@ -151,7 +159,10 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
                             }
                         }
 
-                        return srv.readerMoveCount() > readMoveCnt && srv.writerMoveCount() > writeMoveCnt;
+                        if (usePairedConnections())
+                            return srv.readerMoveCount() > readMoveCnt && srv.writerMoveCount() > writeMoveCnt;
+                        else
+                            return srv.readerMoveCount() > readMoveCnt || srv.writerMoveCount() > writeMoveCnt;
                     }
                 }, 30_000);
 
@@ -165,8 +176,12 @@ public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
                     ", rc2=" + readMoveCnt2 +
                     ", wc2=" + writeMoveCnt2 + ']');
 
-                assertTrue(readMoveCnt2 > readMoveCnt1);
-                assertTrue(writeMoveCnt2 > writeMoveCnt1);
+                if (usePairedConnections()) {
+                    assertTrue(readMoveCnt2 > readMoveCnt1);
+                    assertTrue(writeMoveCnt2 > writeMoveCnt1);
+                }
+                else
+                    assertTrue(readMoveCnt2 > readMoveCnt1 || writeMoveCnt2 > writeMoveCnt1);
 
                 readMoveCnt1 = readMoveCnt2;
                 writeMoveCnt1 = writeMoveCnt2;

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
deleted file mode 100644
index 71772ef..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
+++ /dev/null
@@ -1,47 +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.distributed;
-
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
-
-/**
- *
- */
-public class IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi();
-
-        assertTrue(commSpi.isUsePairedConnections());
-
-        commSpi.setUsePairedConnections(false);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryPairedConnectionsTest.java
new file mode 100644
index 0000000..dffb966
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryPairedConnectionsTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecoveryPairedConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi();
+
+        assertFalse(commSpi.isUsePairedConnections());
+
+        commSpi.setUsePairedConnections(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/da5b68cc/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 1e73e79..092d95e 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -40,6 +40,7 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalancePairedConnectionsTest;
 import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
 import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
 import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
@@ -134,7 +135,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicNearUp
 import org.apache.ignite.internal.processors.cache.distributed.CacheTxNearUpdateTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEntrySetIterationPreloadingSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecovery10ConnectionsTest;
-import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryPairedConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecovery10ConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecoveryTest;
@@ -301,7 +302,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class);
         suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class);
         suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class);
-        suite.addTestSuite(IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.class);
+        suite.addTestSuite(IgniteCacheAtomicMessageRecoveryPairedConnectionsTest.class);
         suite.addTestSuite(IgniteCacheAtomicMessageRecovery10ConnectionsTest.class);
         suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class);
         suite.addTestSuite(IgniteCacheMessageWriteTimeoutTest.class);
@@ -339,6 +340,7 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteVariousConnectionNumberTest.class);
         suite.addTestSuite(IgniteCommunicationBalanceTest.class);
+        suite.addTestSuite(IgniteCommunicationBalancePairedConnectionsTest.class);
         suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class);
         suite.addTestSuite(IgniteIoTestMessagesTest.class);
 


[03/50] [abbrv] ignite git commit: IGNITE-4167: Changed IGNITE_TO_STRING_INCLUDE_SENSITIVE default value to "true".

Posted by sb...@apache.org.
IGNITE-4167: Changed IGNITE_TO_STRING_INCLUDE_SENSITIVE default value to "true".


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 6c38eb28623271a3604ee8d966deb88677a3adb1
Parents: 5494dfb
Author: devozerov <vo...@gridgain.com>
Authored: Thu Dec 29 12:20:20 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 29 12:20:20 2016 +0300

----------------------------------------------------------------------
 .../util/tostring/GridToStringBuilder.java        | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6c38eb28/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index 333f95e..6807b3f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -17,6 +17,13 @@
 
 package org.apache.ignite.internal.util.tostring;
 
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.jetbrains.annotations.Nullable;
+
 import java.io.Externalizable;
 import java.io.InputStream;
 import java.io.OutputStream;
@@ -34,12 +41,8 @@ import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.internal.SB;
-import org.apache.ignite.internal.util.typedef.internal.U;
-import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE;
 
 /**
  * Provides auto-generation framework for {@code toString()} output.
@@ -87,7 +90,8 @@ public class GridToStringBuilder {
     public static final int MAX_COL_SIZE = 100;
 
     /** {@link IgniteSystemProperties#IGNITE_TO_STRING_INCLUDE_SENSITIVE} */
-    public static final boolean INCLUDE_SENSITIVE = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_TO_STRING_INCLUDE_SENSITIVE, false);
+    public static final boolean INCLUDE_SENSITIVE =
+        IgniteSystemProperties.getBoolean(IGNITE_TO_STRING_INCLUDE_SENSITIVE, true);
 
     /** */
     private static ThreadLocal<Queue<GridToStringThreadLocal>> threadCache = new ThreadLocal<Queue<GridToStringThreadLocal>>() {


[10/50] [abbrv] ignite git commit: IGNITE-4516: Hadoop: implemented optional compression for remote shuffle output. This closes #1399.

Posted by sb...@apache.org.
IGNITE-4516: Hadoop: implemented optional compression for remote shuffle output. This closes #1399.


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

Branch: refs/heads/ignite-gg-11810-1
Commit: d62542b9bbfff5a221915f2bd1d23ecfee9814cf
Parents: 2774d87
Author: devozerov <vo...@gridgain.com>
Authored: Thu Jan 5 11:30:42 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 5 11:30:42 2017 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopJobProperty.java    |  7 +++
 .../shuffle/HadoopDirectShuffleMessage.java     | 34 +++++++++++-
 .../hadoop/shuffle/HadoopShuffleJob.java        | 57 +++++++++++++++++---
 .../shuffle/direct/HadoopDirectDataOutput.java  | 14 +++++
 .../direct/HadoopDirectDataOutputContext.java   | 48 +++++++++++++++--
 .../direct/HadoopDirectDataOutputState.java     | 14 ++++-
 .../hadoop/impl/HadoopTeraSortTest.java         | 32 +++++++++--
 7 files changed, 188 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 4398acd..4dd3bf5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -103,6 +103,13 @@ public enum HadoopJobProperty {
     SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
 
     /**
+     * Whether shuffle message should be compressed with GZIP.
+     * <p>
+     * Defaults to {@code false}.
+     */
+    SHUFFLE_MSG_GZIP("ignite.shuffle.message.gzip"),
+
+    /**
      * Whether to stripe mapper output for remote reducers.
      * <p>
      * Defaults to {@code false}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
index e81dc5f..f596100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
@@ -57,6 +57,9 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
     @GridDirectTransient
     private transient int bufLen;
 
+    /** Data length. */
+    private int dataLen;
+
     /**
      * Default constructor.
      */
@@ -72,8 +75,9 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
      * @param cnt Count.
      * @param buf Buffer.
      * @param bufLen Buffer length.
+     * @param dataLen Data length.
      */
-    public HadoopDirectShuffleMessage(HadoopJobId jobId, int reducer, int cnt, byte[] buf, int bufLen) {
+    public HadoopDirectShuffleMessage(HadoopJobId jobId, int reducer, int cnt, byte[] buf, int bufLen, int dataLen) {
         assert jobId != null;
 
         this.jobId = jobId;
@@ -81,6 +85,7 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
         this.cnt = cnt;
         this.buf = buf;
         this.bufLen = bufLen;
+        this.dataLen = dataLen;
     }
 
     /**
@@ -111,6 +116,13 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
         return buf;
     }
 
+    /**
+     * @return Data length.
+     */
+    public int dataLength() {
+        return dataLen;
+    }
+
     /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
@@ -147,6 +159,12 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
 
                 writer.incrementState();
 
+            case 4:
+                if (!writer.writeInt("dataLen", dataLen))
+                    return false;
+
+                writer.incrementState();
+
         }
 
         return true;
@@ -194,6 +212,14 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
 
                 reader.incrementState();
 
+            case 4:
+                dataLen = reader.readInt("dataLen");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
         }
 
         return reader.afterMessageRead(HadoopDirectShuffleMessage.class);
@@ -206,7 +232,7 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 4;
+        return 5;
     }
 
     /** {@inheritDoc} */
@@ -222,6 +248,8 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
         out.writeInt(cnt);
 
         U.writeByteArray(out, buf);
+
+        out.writeInt(dataLen);
     }
 
     /** {@inheritDoc} */
@@ -234,6 +262,8 @@ public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
 
         buf = U.readByteArray(in);
         bufLen = buf != null ? buf.length : 0;
+
+        dataLen = in.readInt();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 1c546a1..0394865 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -56,6 +56,8 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -63,10 +65,12 @@ import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.zip.GZIPInputStream;
 
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.PARTITION_HASHMAP_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_JOB_THROTTLE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT;
+import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MSG_GZIP;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MSG_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_REDUCER_NO_SORTING;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get;
@@ -79,6 +83,9 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     private static final int DFLT_SHUFFLE_MSG_SIZE = 1024 * 1024;
 
     /** */
+    private static final boolean DFLT_SHUFFLE_MSG_GZIP = false;
+
+    /** */
     private final HadoopJob job;
 
     /** */
@@ -130,6 +137,9 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     /** Message size. */
     private final int msgSize;
 
+    /** Whether to GZIP shuffle messages. */
+    private final boolean msgGzip;
+
     /** Whether to strip mappers for remote execution. */
     private final boolean stripeMappers;
 
@@ -190,6 +200,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
         stripeMappers = stripeMappers0;
 
         msgSize = get(job.info(), SHUFFLE_MSG_SIZE, DFLT_SHUFFLE_MSG_SIZE);
+        msgGzip = get(job.info(), SHUFFLE_MSG_GZIP, DFLT_SHUFFLE_MSG_GZIP);
 
         locReducersCtx = new AtomicReferenceArray<>(totalReducerCnt);
 
@@ -360,22 +371,26 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
      * @throws IgniteCheckedException Exception.
      */
     public void onDirectShuffleMessage(T src, HadoopDirectShuffleMessage msg) throws IgniteCheckedException {
-        assert msg.buffer() != null;
+        byte[] buf = extractBuffer(msg);
 
-        HadoopTaskContext taskCtx = locReducersCtx.get(msg.reducer()).get();
+        assert buf != null;
+
+        int rdc = msg.reducer();
+
+        HadoopTaskContext taskCtx = locReducersCtx.get(rdc).get();
 
         HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(taskCtx.counters(), null);
 
-        perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis());
+        perfCntr.onShuffleMessage(rdc, U.currentTimeMillis());
 
-        HadoopMultimap map = getOrCreateMap(locMaps, msg.reducer());
+        HadoopMultimap map = getOrCreateMap(locMaps, rdc);
 
         HadoopSerialization keySer = taskCtx.keySerialization();
         HadoopSerialization valSer = taskCtx.valueSerialization();
 
         // Add data from message to the map.
         try (HadoopMultimap.Adder adder = map.startAdding(taskCtx)) {
-            HadoopDirectDataInput in = new HadoopDirectDataInput(msg.buffer());
+            HadoopDirectDataInput in = new HadoopDirectDataInput(buf);
 
             Object key = null;
             Object val = null;
@@ -393,6 +408,31 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     }
 
     /**
+     * Extract buffer from direct shuffle message.
+     *
+     * @param msg Message.
+     * @return Buffer.
+     */
+    private byte[] extractBuffer(HadoopDirectShuffleMessage msg) throws IgniteCheckedException {
+        if (msgGzip) {
+            byte[] res = new byte[msg.dataLength()];
+
+            try (GZIPInputStream in = new GZIPInputStream(new ByteArrayInputStream(msg.buffer()), res.length)) {
+                int len = in.read(res, 0, res.length);
+
+                assert len == res.length;
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to uncompress direct shuffle message.", e);
+            }
+
+            return res;
+        }
+        else
+            return msg.buffer();
+    }
+
+    /**
      * @param ack Shuffle ack.
      */
     @SuppressWarnings("ConstantConditions")
@@ -595,7 +635,8 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
      * @param rmtDirectCtx Remote direct context.
      * @param reset Whether to perform reset.
      */
-    private void sendShuffleMessage(int rmtMapIdx, @Nullable HadoopDirectDataOutputContext rmtDirectCtx, boolean reset) {
+    private void sendShuffleMessage(int rmtMapIdx, @Nullable HadoopDirectDataOutputContext rmtDirectCtx,
+        boolean reset) {
         if (rmtDirectCtx == null)
             return;
 
@@ -612,7 +653,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
             rmtDirectCtx.reset();
 
         HadoopDirectShuffleMessage msg = new HadoopDirectShuffleMessage(job.id(), rmtRdcIdx, cnt,
-            state.buffer(), state.bufferLength());
+            state.buffer(), state.bufferLength(), state.dataLength());
 
         T nodeId = reduceAddrs[rmtRdcIdx];
 
@@ -983,7 +1024,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
                     HadoopDirectDataOutputContext rmtDirectCtx = rmtDirectCtxs[idx];
 
                     if (rmtDirectCtx == null) {
-                        rmtDirectCtx = new HadoopDirectDataOutputContext(msgSize, taskCtx);
+                        rmtDirectCtx = new HadoopDirectDataOutputContext(msgSize, msgGzip, taskCtx);
 
                         rmtDirectCtxs[idx] = rmtDirectCtx;
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
index 151e552..5840994 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
@@ -170,6 +170,13 @@ public class HadoopDirectDataOutput extends OutputStream implements DataOutput {
     }
 
     /**
+     * @return Buffer length (how much memory is allocated).
+     */
+    public int bufferLength() {
+        return bufSize;
+    }
+
+    /**
      * @return Position.
      */
     public int position() {
@@ -184,6 +191,13 @@ public class HadoopDirectDataOutput extends OutputStream implements DataOutput {
     }
 
     /**
+     * Reset the stream.
+     */
+    public void reset() {
+        pos = 0;
+    }
+
+    /**
      * Ensure that the given amount of bytes is available within the stream, then shift the position.
      *
      * @param cnt Count.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
index bc70ef3..454278b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
@@ -18,16 +18,29 @@
 package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
 
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 
+import java.io.IOException;
+import java.util.zip.GZIPOutputStream;
+
 /**
  * Hadoop data output context for direct communication.
  */
 public class HadoopDirectDataOutputContext {
+    /** Initial allocation size for GZIP output. We start with very low value, but then it will grow if needed. */
+    private static final int GZIP_OUT_MIN_ALLOC_SIZE = 1024;
+
+    /** GZIP buffer size. We should remove it when we implement efficient direct GZIP output. */
+    private static final int GZIP_BUFFER_SIZE = 8096;
+
     /** Flush size. */
     private final int flushSize;
 
+    /** Whether to perform GZIP. */
+    private final boolean gzip;
+
     /** Key serialization. */
     private final HadoopSerialization keySer;
 
@@ -37,6 +50,9 @@ public class HadoopDirectDataOutputContext {
     /** Data output. */
     private HadoopDirectDataOutput out;
 
+    /** Data output for GZIP. */
+    private HadoopDirectDataOutput gzipOut;
+
     /** Number of keys written. */
     private int cnt;
 
@@ -44,17 +60,22 @@ public class HadoopDirectDataOutputContext {
      * Constructor.
      *
      * @param flushSize Flush size.
+     * @param gzip Whether to perform GZIP.
      * @param taskCtx Task context.
      * @throws IgniteCheckedException If failed.
      */
-    public HadoopDirectDataOutputContext(int flushSize, HadoopTaskContext taskCtx)
+    public HadoopDirectDataOutputContext(int flushSize, boolean gzip, HadoopTaskContext taskCtx)
         throws IgniteCheckedException {
         this.flushSize = flushSize;
+        this.gzip = gzip;
 
         keySer = taskCtx.keySerialization();
         valSer = taskCtx.valueSerialization();
 
         out = new HadoopDirectDataOutput(flushSize);
+
+        if (gzip)
+            gzipOut = new HadoopDirectDataOutput(Math.max(flushSize / 8, GZIP_OUT_MIN_ALLOC_SIZE));
     }
 
     /**
@@ -85,16 +106,35 @@ public class HadoopDirectDataOutputContext {
      * @return State.
      */
     public HadoopDirectDataOutputState state() {
-        return new HadoopDirectDataOutputState(out.buffer(), out.position());
+        if (gzip) {
+            try {
+                try (GZIPOutputStream gzip = new GZIPOutputStream(gzipOut, GZIP_BUFFER_SIZE)) {
+                    gzip.write(out.buffer(), 0, out.position());
+                }
+
+                return new HadoopDirectDataOutputState(gzipOut.buffer(), gzipOut.position(), out.position());
+            }
+            catch (IOException e) {
+                throw new IgniteException("Failed to compress.", e);
+            }
+        }
+        else
+            return new HadoopDirectDataOutputState(out.buffer(), out.position(), out.position());
     }
 
     /**
      * Reset buffer.
      */
     public void reset() {
-        int allocSize = Math.max(flushSize, out.position());
+        if (gzip) {
+            // In GZIP mode we do not expose normal output to the outside. Hence, no need for reallocation, just reset.
+            out.reset();
+
+            gzipOut = new HadoopDirectDataOutput(gzipOut.bufferLength());
+        }
+        else
+            out = new HadoopDirectDataOutput(flushSize, out.bufferLength());
 
-        out = new HadoopDirectDataOutput(flushSize, allocSize);
         cnt = 0;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
index a9c12e3..cadde7a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
@@ -27,15 +27,20 @@ public class HadoopDirectDataOutputState {
     /** Buffer length. */
     private final int bufLen;
 
+    /** Data length. */
+    private final int dataLen;
+
     /**
      * Constructor.
      *
      * @param buf Buffer.
      * @param bufLen Buffer length.
+     * @param dataLen Data length.
      */
-    public HadoopDirectDataOutputState(byte[] buf, int bufLen) {
+    public HadoopDirectDataOutputState(byte[] buf, int bufLen, int dataLen) {
         this.buf = buf;
         this.bufLen = bufLen;
+        this.dataLen = dataLen;
     }
 
     /**
@@ -51,4 +56,11 @@ public class HadoopDirectDataOutputState {
     public int bufferLength() {
         return bufLen;
     }
+
+    /**
+     * @return Original data length.
+     */
+    public int dataLength() {
+        return dataLen;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d62542b9/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
index b1fa91f..d237180 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
@@ -137,8 +137,10 @@ public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
 
     /**
      * Does actual test TeraSort job Through Ignite API
+     *
+     * @param gzip Whether to use GZIP.
      */
-    protected final void teraSort() throws Exception {
+    protected final void teraSort(boolean gzip) throws Exception {
         System.out.println("TeraSort ===============================================================");
 
         getFileSystem().delete(new Path(sortOutDir), true);
@@ -164,6 +166,10 @@ public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
         jobConf.set("mapred.max.split.size", String.valueOf(splitSize));
 
         jobConf.setBoolean(HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT.propertyName(), true);
+        jobConf.setInt(HadoopJobProperty.SHUFFLE_MSG_SIZE.propertyName(), 4096);
+
+        if (gzip)
+            jobConf.setBoolean(HadoopJobProperty.SHUFFLE_MSG_GZIP.propertyName(), true);
 
         jobConf.set(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(),
             TextPartiallyRawComparator.class.getName());
@@ -347,12 +353,32 @@ public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
 
     /**
      * Runs generate/sort/validate phases of the terasort sample.
-     * @throws Exception
+     *
+     * @throws Exception If failed.
      */
     public void testTeraSort() throws Exception {
+        checkTeraSort(false);
+    }
+
+    /**
+     * Runs generate/sort/validate phases of the terasort sample.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTeraSortGzip() throws Exception {
+        checkTeraSort(true);
+    }
+
+    /**
+     * Check terasort.
+     *
+     * @param gzip GZIP flag.
+     * @throws Exception If failed.
+     */
+    private void checkTeraSort(boolean gzip) throws Exception {
         teraGenerate();
 
-        teraSort();
+        teraSort(gzip);
 
         teraValidate();
     }


[25/50] [abbrv] ignite git commit: .NET: Improve exception messages for binary misconfiguration and missing assemblies

Posted by sb...@apache.org.
.NET: Improve exception messages for binary misconfiguration and missing assemblies


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

Branch: refs/heads/ignite-gg-11810-1
Commit: e304b48b92fe75027442b7f7cfb46b421d58ae49
Parents: d10946b
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Jan 13 16:49:48 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Jan 13 16:49:48 2017 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core/Impl/Binary/BinaryReader.cs  | 16 ++++++++++++----
 1 file changed, 12 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e304b48b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
index 100091f..d9facc3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReader.cs
@@ -697,10 +697,18 @@ namespace Apache.Ignite.Core.Impl.Binary
                     if (desc.Type == null)
                     {
                         if (desc is BinarySurrogateTypeDescriptor)
-                            throw new BinaryObjectException("Unknown type ID: " + hdr.TypeId);
-
-                        throw new BinaryObjectException("No matching type found for object [typeId=" +
-                                                        desc.TypeId + ", typeName=" + desc.TypeName + ']');
+                        {
+                            throw new BinaryObjectException(string.Format(
+                                "Unknown type ID: {0}. " +
+                                "This usually indicates missing BinaryConfiguration." +
+                                "Make sure that all nodes have the same BinaryConfiguration.", hdr.TypeId));
+                        }
+
+                        throw new BinaryObjectException(string.Format(
+                            "No matching type found for object [typeId={0}, typeName={1}]." +
+                            "This usually indicates that assembly with specified type is not loaded on a node." +
+                            "When using Apache.Ignite.exe, make sure to load assemblies with -assembly parameter.",
+                            desc.TypeId, desc.TypeName));
                     }
 
                     // Preserve old frame.


[39/50] [abbrv] ignite git commit: Merge branch ignite-2.0 into ignite-db-x

Posted by sb...@apache.org.
Merge branch ignite-2.0 into ignite-db-x


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 6f258b9dd0c2d3aad6d30e09282da8b953d27a58
Parents: 0ee73fb f136542
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Tue Jan 17 13:14:17 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Tue Jan 17 13:14:17 2017 +0300

----------------------------------------------------------------------
 modules/cloud/pom.xml                           |   6 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../ignite/cache/affinity/AffinityKey.java      |   4 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |   6 +
 .../configuration/HadoopConfiguration.java      |   2 +-
 .../configuration/IgniteConfiguration.java      |  31 +
 .../org/apache/ignite/events/CacheEvent.java    |   6 +-
 .../ignite/events/CacheQueryReadEvent.java      |   8 +-
 .../apache/ignite/hadoop/HadoopInputSplit.java  |  54 ++
 .../org/apache/ignite/hadoop/HadoopJob.java     |  74 +++
 .../ignite/hadoop/HadoopMapReducePlan.java      |  80 +++
 .../ignite/hadoop/HadoopMapReducePlanner.java   |  40 ++
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |  17 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |  28 +
 .../local/LocalIgfsSecondaryFileSystem.java     | 130 +++-
 .../ignite/internal/GridKernalContext.java      |   7 +
 .../ignite/internal/GridKernalContextImpl.java  |  12 +
 .../apache/ignite/internal/IgniteKernal.java    |   3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  19 +
 .../internal/binary/BinaryEnumObjectImpl.java   |  10 +-
 .../ignite/internal/binary/BinaryMetadata.java  |   5 +-
 .../internal/binary/BinaryObjectExImpl.java     |   8 +-
 .../igfs/common/IgfsHandshakeRequest.java       |  17 -
 .../internal/igfs/common/IgfsMarshaller.java    |   2 -
 .../managers/communication/GridIoManager.java   |   2 +
 .../managers/communication/GridIoPolicy.java    |   3 +
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../processors/cache/CacheInvokeResult.java     |   2 +-
 .../processors/cache/CacheLazyEntry.java        |   4 +-
 .../processors/cache/CacheObjectAdapter.java    |   7 +-
 .../processors/cache/GridCacheAdapter.java      |   5 +-
 .../cache/GridCacheMvccCandidate.java           |   9 +-
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   2 +-
 .../GridNearAtomicAbstractUpdateFuture.java     |  34 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  38 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  57 +-
 .../distributed/near/GridNearLockFuture.java    |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |   4 +-
 .../cache/query/GridCacheQueryManager.java      |  13 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +
 .../cache/query/GridCacheSqlQuery.java          |   6 +-
 .../continuous/CacheContinuousQueryEvent.java   |  10 +-
 .../continuous/CacheContinuousQueryManager.java |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  30 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../GridCacheVersionConflictContext.java        |   2 +-
 .../IgniteCacheObjectProcessorImpl.java         |   5 +-
 .../closure/GridClosureProcessor.java           |   6 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastreamer/DataStreamProcessor.java       |  60 +-
 .../datastreamer/DataStreamerImpl.java          |  37 +-
 .../datastructures/CollocatedSetItemKey.java    |   2 +-
 .../GridCacheAtomicLongValue.java               |   2 +
 .../GridCacheAtomicSequenceImpl.java            |   2 +
 .../GridCacheAtomicSequenceValue.java           |   2 +
 .../GridCacheCountDownLatchValue.java           |   3 +
 .../datastructures/GridCacheSetItemKey.java     |   2 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |   4 +-
 .../processors/hadoop/HadoopFileBlock.java      |   1 +
 .../processors/hadoop/HadoopInputSplit.java     |  54 --
 .../internal/processors/hadoop/HadoopJob.java   | 107 ---
 .../internal/processors/hadoop/HadoopJobEx.java | 140 ++++
 .../processors/hadoop/HadoopJobInfo.java        |  54 +-
 .../processors/hadoop/HadoopJobProperty.java    |   9 +-
 .../processors/hadoop/HadoopMapReducePlan.java  |  80 ---
 .../hadoop/HadoopMapReducePlanner.java          |  40 --
 .../processors/hadoop/HadoopTaskContext.java    |   6 +-
 .../processors/hadoop/HadoopTaskInfo.java       |   1 +
 .../hadoop/counter/HadoopCounterWriter.java     |   4 +-
 .../shuffle/HadoopDirectShuffleMessage.java     |  34 +-
 .../processors/igfs/IgfsBaseBlockKey.java       |  42 ++
 .../internal/processors/igfs/IgfsBlockKey.java  |  26 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |  55 ++
 .../processors/igfs/IgfsDataManager.java        |  12 +-
 .../internal/processors/igfs/IgfsImpl.java      |  26 +-
 .../processors/igfs/IgfsIpcHandler.java         |   4 -
 .../processors/igfs/IgfsKernalContextAware.java |  32 -
 .../processors/igfs/IgfsMetaManager.java        |   2 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |  10 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |  40 --
 .../local/LocalFileSystemBlockKey.java          | 103 +++
 .../local/LocalFileSystemIgfsFile.java          |  10 +-
 .../LocalFileSystemPositionedReadable.java      |  65 ++
 ...fsSecondaryFileSystemPositionedReadable.java |  65 --
 .../internal/processors/job/GridJobWorker.java  |   7 +-
 .../odbc/OdbcQueryExecuteRequest.java           |   6 +-
 .../platform/PlatformNativeException.java       |   3 +-
 .../internal/processors/pool/PoolProcessor.java |   5 +
 .../processors/query/GridQueryProcessor.java    |  37 +-
 .../processors/resource/GridResourceIoc.java    |   6 +-
 .../resource/GridResourceProcessor.java         |  31 +-
 .../processors/rest/GridRestResponse.java       |   2 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 159 ++++-
 .../util/tostring/GridToStringBuilder.java      | 656 +++++++++++++++++--
 .../util/tostring/GridToStringInclude.java      |  12 +-
 .../util/tostring/GridToStringThreadLocal.java  |  12 +-
 .../query/VisorQueryScanSubstringFilter.java    |   5 +-
 .../internal/visor/query/VisorQueryUtils.java   |  60 ++
 .../ignite/resources/FileSystemResource.java    |  62 ++
 .../communication/tcp/TcpCommunicationSpi.java  |  20 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   5 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   3 +
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  16 +-
 .../resources/META-INF/classnames.properties    |   5 +-
 .../ignite/cache/store/jdbc/model/Person.java   |   2 +-
 ...mmunicationBalancePairedConnectionsTest.java |  28 +
 .../IgniteCommunicationBalanceTest.java         |  25 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |   6 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 --
 ...micMessageRecoveryPairedConnectionsTest.java |  47 ++
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |   5 +-
 .../AtomicPutAllChangingTopologyTest.java       | 212 ++++++
 .../cache/query/IndexingSpiQuerySelfTest.java   | 199 +++++-
 .../DataStreamProcessorSelfTest.java            | 104 +++
 .../processors/igfs/IgfsAbstractSelfTest.java   |  18 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |  26 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |   5 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |  81 +++
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |   7 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java | 270 ++++++++
 .../tostring/GridToStringBuilderSelfTest.java   |  33 +-
 .../junits/GridTestKernalContext.java           |  12 +-
 .../IgniteCacheFailoverTestSuite.java           |   3 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |   3 +
 .../apache/ignite/stream/flume/IgniteSink.java  |   7 +-
 modules/gce/pom.xml                             |   4 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   6 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  23 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java | 416 ------------
 .../IgniteHadoopWeightedMapReducePlanner.java   |  15 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 118 ++++
 .../planner/HadoopTestRoundRobinMrPlanner.java  |  75 +++
 .../processors/hadoop/HadoopCommonUtils.java    |   1 +
 .../processors/hadoop/HadoopContext.java        |   2 +
 .../processors/hadoop/HadoopExternalSplit.java  |   1 +
 .../processors/hadoop/HadoopProcessor.java      |   4 +-
 .../processors/hadoop/HadoopSplitWrapper.java   |   1 +
 .../HadoopFileSystemCounterWriterDelegate.java  |   4 +-
 .../HadoopIgfsSecondaryFileSystemDelegate.java  |   4 +-
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |  38 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |   4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |  61 +-
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   9 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |  24 +-
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  33 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |   6 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |   4 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |   2 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |  15 +-
 .../hadoop/impl/v2/HadoopV2Splitter.java        |   2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  70 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |   4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |  32 +-
 .../planner/HadoopAbstractMapReducePlanner.java | 116 ----
 .../planner/HadoopDefaultMapReducePlan.java     |   4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |   4 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  66 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |  75 ++-
 .../shuffle/direct/HadoopDirectDataOutput.java  |  14 +
 .../direct/HadoopDirectDataOutputContext.java   |  48 +-
 .../direct/HadoopDirectDataOutputState.java     |  14 +-
 .../shuffle/streams/HadoopDataInStream.java     |  34 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |  18 +
 .../HadoopEmbeddedTaskExecutor.java             |   8 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   6 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |   8 +-
 .../external/HadoopExternalTaskExecutor.java    |  16 +-
 .../child/HadoopChildProcessRunner.java         |   4 +-
 .../resources/META-INF/classnames.properties    |   8 +
 .../impl/HadoopAbstractMapReduceTest.java       |   2 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   3 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |   4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  | 619 -----------------
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   1 +
 .../hadoop/impl/HadoopPlannerMockJob.java       |  10 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |  19 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |  16 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |   4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |   4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  32 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |  75 ---
 .../hadoop/impl/HadoopTestTaskContext.java      |   6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |   6 +-
 .../HadoopWeightedMapReducePlannerTest.java     |   6 +-
 .../client/HadoopClientProtocolSelfTest.java    |   2 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   7 +-
 .../impl/igfs/Hadoop1OverIgfsProxyTest.java     |  67 ++
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   6 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |   2 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |   2 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   6 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   4 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  32 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   6 +-
 ...condaryFileSystemInitializationSelfTest.java |   4 +-
 .../collections/HadoopAbstractMapTest.java      |   4 +-
 .../streams/HadoopDataStreamSelfTest.java       | 177 ++++-
 .../testsuites/IgniteHadoopTestSuite.java       |   4 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |   6 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |  18 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |  14 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   4 +-
 .../org/apache/ignite/logger/jcl/JclLogger.java |   9 +-
 modules/platforms/cpp/binary/Makefile.am        |   2 +-
 .../ignite/impl/binary/binary_type_manager.h    |   2 +-
 .../src/impl/binary/binary_type_manager.cpp     |   6 +-
 .../binary/src/impl/interop/interop_memory.cpp  |   2 +-
 modules/platforms/cpp/common/Makefile.am        |   2 +-
 .../include/ignite/common/dynamic_size_array.h  |   2 +-
 .../cpp/common/project/vs/common.vcxproj        |   1 +
 modules/platforms/cpp/configure.ac              |   2 +-
 modules/platforms/cpp/configure.acrel           |   2 +-
 modules/platforms/cpp/core-test/Makefile.am     |   2 +-
 .../cpp/core-test/src/cache_query_test.cpp      |   8 +-
 .../cpp/core-test/src/reference_test.cpp        |  14 +-
 modules/platforms/cpp/core/Makefile.am          |   2 +-
 .../cache/query/continuous/continuous_query.h   |   8 +-
 .../include/ignite/cache/query/query_cursor.h   |   4 +-
 .../query/continuous/continuous_query_impl.h    |   8 +-
 .../core/include/ignite/impl/handle_registry.h  |   2 +-
 .../cpp/core/include/ignite/impl/operations.h   |   4 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   5 +-
 .../cpp/core/src/impl/handle_registry.cpp       |   2 +-
 modules/platforms/cpp/examples/configure.ac     |   2 +-
 .../cpp/examples/odbc-example/Makefile.am       |   2 +-
 .../cpp/examples/putget-example/Makefile.am     |   2 +-
 .../cpp/examples/query-example/Makefile.am      |   2 +-
 modules/platforms/cpp/ignite/Makefile.am        |   2 +-
 modules/platforms/cpp/jni/Makefile.am           |   2 +-
 .../platforms/cpp/jni/project/vs/jni.vcxproj    |   1 +
 modules/platforms/cpp/jni/src/java.cpp          |   2 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |   2 +-
 .../cpp/odbc-test/config/queries-default.xml    |   1 +
 .../odbc-test/include/sql_test_suite_fixture.h  |   3 +
 .../platforms/cpp/odbc-test/include/test_type.h |  20 +-
 .../cpp/odbc-test/src/api_robustness_test.cpp   | 113 +++-
 .../platforms/cpp/odbc-test/src/column_test.cpp | 189 ++++++
 .../odbc-test/src/sql_test_suite_fixture.cpp    |  17 +
 .../cpp/odbc-test/src/sql_types_test.cpp        | 131 ++++
 modules/platforms/cpp/odbc/Makefile.am          |   5 +-
 modules/platforms/cpp/odbc/include/Makefile.am  |   3 +-
 .../cpp/odbc/include/ignite/odbc/common_types.h |  29 +-
 .../ignite/odbc/config/connection_info.h        |   2 -
 .../cpp/odbc/include/ignite/odbc/log.h          | 123 ++++
 .../odbc/include/ignite/odbc/meta/column_meta.h |   2 -
 .../cpp/odbc/include/ignite/odbc/statement.h    | 143 +++-
 .../cpp/odbc/include/ignite/odbc/utility.h      |  26 +-
 .../cpp/odbc/install/ignite-odbc-amd64.wxs      |   2 +-
 .../cpp/odbc/install/ignite-odbc-x86.wxs        |   2 +-
 .../odbc/os/linux/src/system/socket_client.cpp  |   9 +-
 .../odbc/os/win/src/system/socket_client.cpp    |   9 +-
 .../src/system/ui/dsn_configuration_window.cpp  |  18 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |  17 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |   2 +
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |   6 +
 modules/platforms/cpp/odbc/src/column.cpp       |  28 +-
 .../cpp/odbc/src/config/connection_info.cpp     |   9 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 .../odbc/src/diagnostic/diagnosable_adapter.cpp |   3 +-
 .../odbc/src/diagnostic/diagnostic_record.cpp   |  38 ++
 modules/platforms/cpp/odbc/src/entry_points.cpp |  52 +-
 modules/platforms/cpp/odbc/src/log.cpp          |  83 +++
 .../platforms/cpp/odbc/src/meta/column_meta.cpp |   4 -
 modules/platforms/cpp/odbc/src/odbc.cpp         | 286 ++++----
 .../odbc/src/query/column_metadata_query.cpp    |  13 +-
 .../platforms/cpp/odbc/src/query/data_query.cpp |  23 +-
 .../cpp/odbc/src/query/table_metadata_query.cpp |  12 +-
 modules/platforms/cpp/odbc/src/statement.cpp    | 187 +++++-
 modules/platforms/cpp/odbc/src/utility.cpp      |  29 +-
 .../Apache.Ignite.AspNet.Tests.csproj           |   7 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.AspNet.csproj                 |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Benchmarks.csproj             |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.TestDll.csproj     |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests.csproj             |   7 +
 .../Cache/CacheAbstractTest.cs                  | 627 +-----------------
 .../Cache/CacheAbstractTransactionalTest.cs     | 565 ++++++++++++++++
 .../Cache/CacheLocalAtomicTest.cs               |   5 -
 .../Cache/CacheLocalTest.cs                     |   6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |   5 -
 .../Cache/CachePartitionedAtomicTest.cs         |   5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   7 +-
 .../Cache/CachePartitionedTest.cs               |   7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |   5 -
 .../Cache/CacheReplicatedTest.cs                |   7 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../Log/ConcurrentMemoryTarget.cs               |  73 +++
 .../Log/NLogLoggerTest.cs                       |   5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |   6 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |   5 +
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   1 -
 .../Apache.Ignite.Core.csproj                   |  16 +-
 .../Impl/Binary/BinaryReader.cs                 |  16 +-
 .../Impl/Transactions/TransactionsImpl.cs       |  18 +
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   4 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Transactions/ITransactions.cs               |  19 +-
 .../Apache.Ignite.EntityFramework.Tests.csproj  |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.EntityFramework.csproj        |   9 +-
 .../Impl/ArrayDbDataReader.cs                   |   5 +
 .../Impl/DataReaderResult.cs                    |   2 +
 .../Impl/DbCommandInfo.cs                       |  11 +
 .../Impl/DbCommandProxy.cs                      |   1 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Linq.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.Log4Net.csproj                |   5 +
 .../IgniteLog4NetLogger.cs                      |  16 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.NLog.csproj                   |   5 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |   3 +
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   5 +
 .../Apache.Ignite/Properties/AssemblyInfo.cs    |   4 +-
 modules/platforms/dotnet/README.md              |   6 +-
 modules/platforms/dotnet/build.ps1              |   2 +-
 .../Apache.Ignite.Examples.csproj               |   3 +
 .../examples/Apache.Ignite.Examples/App.config  |   2 +
 .../DataStructures/AtomicSequenceExample.cs     |  26 +-
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Apache.Ignite.ExamplesDll.csproj            |   4 +
 .../Properties/AssemblyInfo.cs                  |   4 +-
 .../Services/MapService.cs                      |   2 -
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |  11 +-
 modules/web-console/backend/app/mongo.js        |  19 +
 .../generator/AbstractTransformer.js            |   5 +
 .../modules/configuration/generator/Beans.js    |   4 +
 .../generator/ConfigurationGenerator.js         |  36 +
 .../states/configuration/caches/affinity.jade   |  82 +++
 .../states/configuration/caches/memory.jade     |   4 +-
 .../frontend/views/configuration/caches.jade    |   1 +
 .../yardstick/cache/IgniteIoTestBenchmark.java  |  73 ---
 parent/pom.xml                                  |   2 +-
 352 files changed, 6795 insertions(+), 3698 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
index 7f3f5e4,9fd6bc1..a17f755
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryEnumObjectImpl.java
@@@ -29,9 -29,9 +29,10 @@@ import org.apache.ignite.binary.BinaryO
  import org.apache.ignite.binary.BinaryType;
  import org.apache.ignite.internal.GridDirectTransient;
  import org.apache.ignite.internal.processors.cache.CacheObject;
 +import org.apache.ignite.internal.processors.cache.CacheObjectAdapter;
  import org.apache.ignite.internal.processors.cache.CacheObjectContext;
  import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
+ import org.apache.ignite.internal.util.typedef.internal.S;
  import org.apache.ignite.internal.util.typedef.internal.SB;
  import org.apache.ignite.internal.util.typedef.internal.U;
  import org.apache.ignite.plugin.extensions.communication.MessageReader;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryMetadata.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryObjectExImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
index a34e98d,09a5524..1f13c6f
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheObjectAdapter.java
@@@ -22,9 -22,9 +22,10 @@@ import java.io.IOException
  import java.io.ObjectInput;
  import java.io.ObjectOutput;
  import java.nio.ByteBuffer;
 +import org.apache.ignite.IgniteCheckedException;
  import org.apache.ignite.internal.GridDirectTransient;
  import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+ import org.apache.ignite.internal.util.typedef.internal.S;
  import org.apache.ignite.internal.util.typedef.internal.U;
  import org.apache.ignite.plugin.extensions.communication.MessageReader;
  import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@@ -142,70 -120,8 +143,72 @@@ public abstract class CacheObjectAdapte
  
      /** {@inheritDoc} */
      public String toString() {
-         return getClass().getSimpleName() + " [val=" + val + ", hasValBytes=" + (valBytes != null) + ']';
+         return S.toString(S.INCLUDE_SENSITIVE ? getClass().getSimpleName() : "CacheObject",
+             "val", val, true,
+             "hasValBytes", valBytes != null, false);
      }
 +
 +    /**
 +     * @param dataLen Serialized value length.
 +     * @return Full size required to store cache object.
 +     * @see #putValue(byte, ByteBuffer, int, int, byte[], int)
 +     */
 +    public static int objectPutSize(int dataLen) {
 +        return dataLen + 5;
 +    }
 +
 +    /**
 +     * @param cacheObjType Cache object type.
 +     * @param buf Buffer to write value to.
 +     * @param off Offset in source binary data.
 +     * @param len Length of the data to write.
 +     * @param valBytes Binary data.
 +     * @param start Start offset in binary data.
 +     * @return {@code True} if data were successfully written.
 +     * @throws IgniteCheckedException If failed.
 +     */
 +    public static boolean putValue(byte cacheObjType, final ByteBuffer buf, int off, int len,
 +        byte[] valBytes, final int start) throws IgniteCheckedException {
 +        int dataLen = valBytes.length;
 +
 +        if (buf.remaining() < len)
 +            return false;
 +
 +        final int headSize = 5; // 4 bytes len + 1 byte type
 +
 +        if (off == 0 && len >= headSize) {
 +            buf.putInt(dataLen);
 +            buf.put(cacheObjType);
 +
 +            len -= headSize;
 +        }
 +        else if (off >= headSize)
 +            off -= headSize;
 +        else {
 +            // Partial header write.
 +            final ByteBuffer head = ByteBuffer.allocate(headSize);
 +
 +            head.order(buf.order());
 +
 +            head.putInt(dataLen);
 +            head.put(cacheObjType);
 +
 +            head.position(off);
 +
 +            if (len < head.capacity())
 +                head.limit(off + Math.min(len, head.capacity() - off));
 +
 +            buf.put(head);
 +
 +            if (head.limit() < headSize)
 +                return true;
 +
 +            len -= headSize - off;
 +            off = 0;
 +        }
 +
 +        buf.put(valBytes, start + off, len);
 +
 +        return true;
 +    }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicAbstractUpdateFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheSqlQuery.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
index 3922e3e,eddf302..e664554
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryEvent.java
@@@ -99,10 -99,9 +99,10 @@@ class CacheContinuousQueryEvent<K, V> e
      /** {@inheritDoc} */
      @Override public String toString() {
          return S.toString(CacheContinuousQueryEvent.class, this,
-             "evtType", getEventType(),
-             "key", getKey(),
-             "newVal", getValue(),
-             "oldVal", getOldValue(),
-             "partCntr", getPartitionUpdateCounter());
+             "evtType", getEventType(), false,
+             "key", getKey(), true,
+             "newVal", getValue(), true,
 -            "oldVal", getOldValue(), true);
++            "oldVal", getOldValue(), true,
++            "partCntr", getPartitionUpdateCounter(), false);
      }
  }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 253197d,a830186..b3645ea
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@@ -562,11 -562,14 +563,14 @@@ public abstract class GridCacheStoreMan
              if (key instanceof GridCacheInternal)
                  return true;
  
 -            key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
 -            val = cctx.unwrapBinaryIfNeeded(val, !convertBinary());
 +            Object key0 = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
 +            Object val0 = cctx.unwrapBinaryIfNeeded(val, !convertBinary());
  
-             if (log.isDebugEnabled())
-                 log.debug("Storing value in cache store [key=" + key0 + ", val=" + val0 + ']');
+             if (log.isDebugEnabled()) {
+                 log.debug(S.toString("Storing value in cache store",
 -                    "key", key, true,
 -                    "val", val, true));
++                    "key", key0, true,
++                    "val", val0, true));
+             }
  
              sessionInit0(tx);
  
@@@ -590,8 -593,11 +594,11 @@@
                  sessionEnd0(tx, threwEx);
              }
  
-             if (log.isDebugEnabled())
-                 log.debug("Stored value in cache store [key=" + key0 + ", val=" + val0 + ']');
+             if (log.isDebugEnabled()) {
+                 log.debug(S.toString("Stored value in cache store",
 -                    "key", key, true,
 -                    "val", val, true));
++                    "key", key0, true,
++                    "val", val0, true));
+             }
  
              return true;
          }
@@@ -668,10 -671,10 +675,10 @@@
              if (key instanceof GridCacheInternal)
                  return false;
  
 -            key = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
 +            Object key0 = cctx.unwrapBinaryIfNeeded(key, !convertBinary());
  
              if (log.isDebugEnabled())
-                 log.debug("Removing value from cache store [key=" + key0 + ']');
 -                log.debug(S.toString("Removing value from cache store", "key", key, true));
++                log.debug(S.toString("Removing value from cache store", "key", key0, true));
  
              sessionInit0(tx);
  
@@@ -696,7 -699,7 +703,7 @@@
              }
  
              if (log.isDebugEnabled())
-                 log.debug("Removed value from cache store [key=" + key0 + ']');
 -                log.debug(S.toString("Removed value from cache store", "key", key, true));
++                log.debug(S.toString("Removed value from cache store", "key", key0, true));
  
              return true;
          }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index c2692d0,614c612..ca6ef90
--- 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
@@@ -299,9 -242,10 +299,8 @@@ public class IgniteCacheObjectProcessor
      @Override public CacheObjectContext contextForCache(CacheConfiguration ccfg) throws IgniteCheckedException {
          assert ccfg != null;
  
-         boolean storeVal = ctx.config().isPeerClassLoadingEnabled() ||
-             GridQueryProcessor.isEnabled(ccfg) ||
-             !ccfg.isCopyOnRead();
 -        CacheMemoryMode memMode = ccfg.getMemoryMode();
 -
+         boolean storeVal = !ccfg.isCopyOnRead() || (!isBinaryEnabled(ccfg) &&
+             (GridQueryProcessor.isEnabled(ccfg) || ctx.config().isPeerClassLoadingEnabled()));
  
          CacheObjectContext res = new CacheObjectContext(ctx,
              ccfg.getName(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 5200907,3286bac..cf85a52
--- 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
@@@ -1128,16 -1055,15 +1143,18 @@@ public class GridQueryProcessor extend
          if (log.isDebugEnabled())
              log.debug("Remove [space=" + space + ", key=" + key + ", val=" + val + "]");
  
 +        CacheObjectContext coctx = null;
 +
          if (ctx.indexing().enabled()) {
 -            CacheObjectContext coctx = cacheObjectContext(space);
 +            coctx = cacheObjectContext(space);
  
-             ctx.indexing().remove(space, key.value(coctx, false));
+             Object key0 = unwrap(key, coctx);
+ 
+             ctx.indexing().remove(space, key0);
          }
  
 -        if (idx == null)
 +        // If val == null we only need to call SPI.
 +        if (idx == null || val == null)
              return;
  
          if (!busyLock.enterBusy())

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6f258b9d/modules/hadoop/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --cc modules/hadoop/src/main/resources/META-INF/classnames.properties
index 051094d,f528107..d8848ac
--- a/modules/hadoop/src/main/resources/META-INF/classnames.properties
+++ b/modules/hadoop/src/main/resources/META-INF/classnames.properties
@@@ -26,9 -26,20 +26,17 @@@ org.apache.ignite.hadoop.util.KerberosU
  org.apache.ignite.hadoop.util.KerberosUserNameMapper$State
  org.apache.ignite.hadoop.util.UserNameMapper
  org.apache.ignite.internal.processors.hadoop.HadoopAttributes
 -org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo
  org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit
+ org.apache.ignite.internal.processors.hadoop.HadoopFileBlock
+ org.apache.ignite.hadoop.HadoopInputSplit
+ org.apache.ignite.internal.processors.hadoop.HadoopJobId
+ org.apache.ignite.internal.processors.hadoop.HadoopJobInfo
+ org.apache.ignite.internal.processors.hadoop.HadoopJobPhase
+ org.apache.ignite.internal.processors.hadoop.HadoopJobProperty
+ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
+ org.apache.ignite.hadoop.HadoopMapReducePlan
  org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper
  org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException
 -org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo
 -org.apache.ignite.internal.processors.hadoop.HadoopTaskType
  org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterAdapter
  org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl
  org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl$CounterKey


[12/50] [abbrv] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by sb...@apache.org.
Merge branch 'master' into ignite-2.0


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

Branch: refs/heads/ignite-gg-11810-1
Commit: 6090ebdfcd0ea3840b0d32cb10197b43615e1e89
Parents: 1f74346 beb242b
Author: devozerov <vo...@gridgain.com>
Authored: Thu Jan 5 12:23:06 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Jan 5 12:23:06 2017 +0300

----------------------------------------------------------------------
 modules/cloud/pom.xml                           |   6 +-
 .../java/org/apache/ignite/IgniteLogger.java    |   4 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../ignite/cache/affinity/AffinityKey.java      |   4 +-
 .../org/apache/ignite/events/CacheEvent.java    |   6 +-
 .../ignite/events/CacheQueryReadEvent.java      |   8 +-
 .../ignite/internal/binary/BinaryContext.java   |   4 +-
 .../internal/binary/BinaryEnumObjectImpl.java   |  10 +-
 .../ignite/internal/binary/BinaryMetadata.java  |   5 +-
 .../internal/binary/BinaryObjectExImpl.java     |   8 +-
 .../ignite/internal/binary/BinaryTypeProxy.java |  15 +-
 .../ignite/internal/binary/BinaryUtils.java     |   4 +-
 .../cache/CacheInvokeDirectResult.java          |   2 +-
 .../processors/cache/CacheInvokeResult.java     |   2 +-
 .../processors/cache/CacheLazyEntry.java        |   4 +-
 .../processors/cache/CacheObjectAdapter.java    |   7 +-
 .../processors/cache/GridCacheAdapter.java      |   5 +-
 .../cache/GridCacheMvccCandidate.java           |   9 +-
 .../processors/cache/GridCacheReturn.java       |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   2 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   2 +-
 .../distributed/near/GridNearLockFuture.java    |   2 +-
 .../cache/query/GridCacheQueryAdapter.java      |   4 +-
 .../cache/query/GridCacheQueryManager.java      |  13 +-
 .../cache/query/GridCacheQueryRequest.java      |   2 +
 .../cache/query/GridCacheSqlQuery.java          |   4 +-
 .../continuous/CacheContinuousQueryEvent.java   |   8 +-
 .../continuous/CacheContinuousQueryManager.java |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  30 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  11 +-
 .../GridCacheVersionConflictContext.java        |   2 +-
 .../closure/GridClosureProcessor.java           |   4 +-
 .../continuous/GridContinuousMessage.java       |   2 +-
 .../datastructures/CollocatedSetItemKey.java    |   2 +-
 .../GridCacheAtomicLongValue.java               |   2 +
 .../GridCacheAtomicSequenceImpl.java            |   2 +
 .../GridCacheAtomicSequenceValue.java           |   2 +
 .../GridCacheCountDownLatchValue.java           |   3 +
 .../datastructures/GridCacheSetItemKey.java     |   2 +-
 .../processors/hadoop/HadoopJobProperty.java    |   7 +
 .../shuffle/HadoopDirectShuffleMessage.java     |  34 +-
 .../internal/processors/job/GridJobWorker.java  |   7 +-
 .../odbc/OdbcQueryExecuteRequest.java           |   6 +-
 .../platform/PlatformNativeException.java       |   3 +-
 .../processors/query/GridQueryProcessor.java    |  35 +-
 .../processors/rest/GridRestResponse.java       |   2 +-
 .../internal/util/future/GridFutureAdapter.java |   2 +-
 .../util/lang/GridMetadataAwareAdapter.java     |   2 +-
 .../ignite/internal/util/nio/GridNioServer.java | 159 ++++-
 .../util/tostring/GridToStringBuilder.java      | 656 +++++++++++++++++--
 .../util/tostring/GridToStringInclude.java      |  12 +-
 .../util/tostring/GridToStringThreadLocal.java  |  12 +-
 .../query/VisorQueryScanSubstringFilter.java    |   5 +-
 .../internal/visor/query/VisorQueryUtils.java   |  60 ++
 .../communication/tcp/TcpCommunicationSpi.java  |  20 +-
 .../tcp/TcpCommunicationSpiMBean.java           |   5 +-
 .../apache/ignite/spi/indexing/IndexingSpi.java |   3 +
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |  16 +-
 .../resources/META-INF/classnames.properties    |   1 +
 .../internal/binary/BinaryEnumsSelfTest.java    |  18 +
 ...mmunicationBalancePairedConnectionsTest.java |  28 +
 .../IgniteCommunicationBalanceTest.java         |  25 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |   7 +-
 ...cMessageRecoveryNoPairedConnectionsTest.java |  47 --
 ...micMessageRecoveryPairedConnectionsTest.java |  47 ++
 .../cache/query/IndexingSpiQuerySelfTest.java   | 199 +++++-
 .../tostring/GridToStringBuilderSelfTest.java   |  33 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   6 +-
 modules/gce/pom.xml                             |   4 +-
 .../hadoop/impl/igfs/HadoopIgfsJclLogger.java   |   9 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |  57 +-
 .../shuffle/direct/HadoopDirectDataOutput.java  |  14 +
 .../direct/HadoopDirectDataOutputContext.java   |  48 +-
 .../direct/HadoopDirectDataOutputState.java     |  14 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |  32 +-
 .../org/apache/ignite/logger/jcl/JclLogger.java |   9 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Apache.Ignite.Core.Tests.csproj             |   1 +
 .../Cache/CacheAbstractTest.cs                  | 625 ------------------
 .../Cache/CacheAbstractTransactionalTest.cs     | 565 ++++++++++++++++
 .../Cache/CacheLocalAtomicTest.cs               |   5 -
 .../Cache/CacheLocalTest.cs                     |   6 +-
 .../CachePartitionedAtomicNearEnabledTest.cs    |   5 -
 .../Cache/CachePartitionedAtomicTest.cs         |   5 -
 .../Cache/CachePartitionedNearEnabledTest.cs    |   7 +-
 .../Cache/CachePartitionedTest.cs               |   7 +-
 .../Cache/CacheReplicatedAtomicTest.cs          |   5 -
 .../Cache/CacheReplicatedTest.cs                |   7 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |   4 +-
 .../Properties/AssemblyInfo.cs                  |   2 +-
 .../Impl/Transactions/TransactionsImpl.cs       |  18 +
 .../Transactions/ITransactions.cs               |  19 +-
 .../apache/ignite/logger/slf4j/Slf4jLogger.java |  11 +-
 modules/web-console/backend/app/mongo.js        |  19 +
 .../generator/AbstractTransformer.js            |   5 +
 .../modules/configuration/generator/Beans.js    |   4 +
 .../generator/ConfigurationGenerator.js         |  36 +
 .../states/configuration/caches/affinity.jade   |  82 +++
 .../states/configuration/caches/memory.jade     |   4 +-
 .../frontend/views/configuration/caches.jade    |   1 +
 .../yardstick/cache/IgniteIoTestBenchmark.java  |  73 ---
 parent/pom.xml                                  |   2 +-
 104 files changed, 2351 insertions(+), 1038 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6090ebdf/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6090ebdf/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index a3115bf,4dd3bf5..60992d5
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@@ -103,9 -103,16 +103,16 @@@ public enum HadoopJobProperty 
      SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
  
      /**
+      * Whether shuffle message should be compressed with GZIP.
+      * <p>
+      * Defaults to {@code false}.
+      */
+     SHUFFLE_MSG_GZIP("ignite.shuffle.message.gzip"),
+ 
+     /**
       * Whether to stripe mapper output for remote reducers.
       * <p>
 -     * Defaults to {@code false}.
 +     * Defaults to {@code true}.
       */
      SHUFFLE_MAPPER_STRIPED_OUTPUT("ignite.shuffle.mapper.striped.output"),
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/6090ebdf/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------


[46/50] [abbrv] ignite git commit: ignite-3477 PageMemory optimizations - use page address instead of ByteBuffer to work with page memory - got rid of pages pin/unpin - do not copy byte array for cache key comparison - reduced size of data tree search ro

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/7db65ddd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
index aef7217..e91dda9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/database/tree/BPlusTree.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache.database.tree;
 
 import java.io.Externalizable;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -126,15 +125,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return null;
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO io = io(buf);
+                    BPlusIO io = io(pageAddr);
 
                     if (io.isLeaf())
                         return null;
 
-                    int cnt = io.getCount(buf);
+                    int cnt = io.getCount(pageAddr);
 
                     assert cnt >= 0 : cnt;
 
@@ -144,12 +143,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         res = new ArrayList<>(cnt + 1);
 
                         for (int i = 0; i < cnt; i++)
-                            res.add(inner(io).getLeft(buf, i));
+                            res.add(inner(io).getLeft(pageAddr, i));
 
-                        res.add(inner(io).getRight(buf, cnt - 1));
+                        res.add(inner(io).getRight(pageAddr, cnt - 1));
                     }
                     else {
-                        long left = inner(io).getLeft(buf, 0);
+                        long left = inner(io).getLeft(pageAddr, 0);
 
                         res = left == 0 ? Collections.<Long>emptyList() : Collections.singletonList(left);
                     }
@@ -157,7 +156,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res;
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -173,15 +172,15 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return "<Zero>";
 
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
-                    BPlusIO<L> io = io(buf);
+                    BPlusIO<L> io = io(pageAddr);
 
-                    return printPage(io, buf, keys);
+                    return printPage(io, pageAddr, keys);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
             catch (IgniteCheckedException e) {
@@ -192,17 +191,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Get> askNeighbor = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int isBack) {
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int isBack) {
             assert !io.isLeaf(); // Inner page.
 
             boolean back = isBack == TRUE.ordinal();
 
-            long res = doAskNeighbor(io, buf, back);
+            long res = doAskNeighbor(io, pageAddr, back);
 
             if (back) {
                 assert g.getClass() == Remove.class;
 
-                if (io.getForward(buf) != g.backId) // See how g.backId is setup in removeDown for this check.
+                if (io.getForward(pageAddr) != g.backId) // See how g.backId is setup in removeDown for this check.
                     return RETRY;
 
                 g.backId = res;
@@ -219,25 +218,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Get> search = new GetPageHandler<Get>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Get g, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Get g, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != g.fwdId)
+            if (io.getForward(pageAddr) != g.fwdId)
                 return RETRY;
 
             boolean needBackIfRouting = g.backId != 0;
 
             g.backId = 0; // Usually we'll go left down and don't need it.
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, g.row, g.shift);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, g.row, g.shift);
 
             boolean found = idx >= 0;
 
             if (found) { // Found exact match.
                 assert g.getClass() != GetCursor.class;
 
-                if (g.found(io, buf, idx, lvl))
+                if (g.found(io, pageAddr, idx, lvl))
                     return FOUND;
 
                 // Else we need to reach leaf page, go left down.
@@ -245,20 +244,20 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             else {
                 idx = fix(idx);
 
-                if (g.notFound(io, buf, idx, lvl)) // No way down, stop here.
+                if (g.notFound(io, pageAddr, idx, lvl)) // No way down, stop here.
                     return NOT_FOUND;
             }
 
             assert !io.isLeaf();
 
             // If idx == cnt then we go right down, else left down: getLeft(cnt) == getRight(cnt - 1).
-            g.pageId = inner(io).getLeft(buf, idx);
+            g.pageId = inner(io).getLeft(pageAddr, idx);
 
             // If we see the tree in consistent state, then our right down page must be forward for our left down page,
             // we need to setup fwdId and/or backId to be able to check this invariant on lower level.
             if (idx < cnt) {
                 // Go left down here.
-                g.fwdId = inner(io).getRight(buf, idx);
+                g.fwdId = inner(io).getRight(pageAddr, idx);
             }
             else {
                 // Go right down here or it is an empty branch.
@@ -267,7 +266,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 // Here child's forward is unknown to us (we either go right or it is an empty "routing" page),
                 // need to ask our forward about the child's forward (it must be leftmost child of our forward page).
                 // This is ok from the locking standpoint because we take all locks in the forward direction.
-                long fwdId = io.getForward(buf);
+                long fwdId = io.getForward(pageAddr);
 
                 // Setup fwdId.
                 if (fwdId == 0)
@@ -282,7 +281,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Setup backId.
                 if (cnt != 0) // It is not a routing page and we are going to the right, can get backId here.
-                    g.backId = inner(io).getLeft(buf, cnt - 1);
+                    g.backId = inner(io).getLeft(pageAddr, cnt - 1);
                 else if (needBackIfRouting) {
                     // Can't get backId here because of possible deadlock and it is only needed for remove operation.
                     return GO_DOWN_X;
@@ -295,16 +294,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Put> replace = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             // Check the triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
             assert p.btmLvl == 0 : "split is impossible with replace";
 
-            final int cnt = io.getCount(buf);
-            final int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            final int cnt = io.getCount(pageAddr);
+            final int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx < 0) // Not found, split or merge happened.
                 return RETRY;
@@ -318,7 +317,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert p.oldRow == null;
 
                 // Get old row in leaf page to reduce contention at upper level.
-                p.oldRow = getRow(io, buf, idx);
+                p.oldRow = getRow(io, pageAddr, idx);
 
                 p.finish();
 
@@ -326,7 +325,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 assert p.needReplaceInner == FALSE || p.needReplaceInner == DONE : p.needReplaceInner;
             }
 
-            io.store(buf, idx, newRow, null);
+            io.store(pageAddr, idx, newRow, null);
 
             if (needWalDeltaRecord(page))
                 wal.log(new ReplaceRecord<>(cacheId, page.id(), io, newRow, null, idx));
@@ -337,16 +336,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Put> insert = new GetPageHandler<Put>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Put p, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Put p, int lvl)
             throws IgniteCheckedException {
             assert p.btmLvl == lvl : "we must always insert at the bottom level: " + p.btmLvl + " " + lvl;
 
             // Check triangle invariant.
-            if (io.getForward(buf) != p.fwdId)
+            if (io.getForward(pageAddr) != p.fwdId)
                 return RETRY;
 
-            int cnt = io.getCount(buf);
-            int idx = findInsertionPoint(io, buf, 0, cnt, p.row, 0);
+            int cnt = io.getCount(pageAddr);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, p.row, 0);
 
             if (idx >= 0) // We do not support concurrent put of the same key.
                 throw new IllegalStateException("Duplicate row in index.");
@@ -354,7 +353,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             idx = fix(idx);
 
             // Do insert.
-            L moveUpRow = p.insert(page, io, buf, idx, lvl);
+            L moveUpRow = p.insert(page, io, pageAddr, idx, lvl);
 
             // Check if split happened.
             if (moveUpRow != null) {
@@ -363,8 +362,8 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // Here forward page can't be concurrently removed because we keep write lock on tail which is the only
                 // page who knows about the forward page, because it was just produced by split.
-                p.rightId = io.getForward(buf);
-                p.tail(page, buf);
+                p.rightId = io.getForward(pageAddr);
+                p.tail(page, pageAddr);
 
                 assert p.rightId != 0;
             }
@@ -376,16 +375,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     };
 
     /** */
-    private final GetPageHandler<Remove> removeFromLeaf = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page leaf, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> rmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override public Result run0(Page leaf, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl == 0 : lvl; // Leaf.
 
-            final int cnt = io.getCount(buf);
+            final int cnt = io.getCount(pageAddr);
 
             assert cnt <= Short.MAX_VALUE: cnt;
 
-            int idx = findInsertionPoint(io, buf, 0, cnt, r.row, 0);
+            int idx = findInsertionPoint(io, pageAddr, 0, cnt, r.row, 0);
 
             if (idx < 0) {
                 if (!r.ceil) // We've found exact match on search but now it's gone.
@@ -401,14 +400,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Need to do inner replace when we remove the rightmost element and the leaf have no forward page,
             // i.e. it is not the rightmost leaf of the tree.
-            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(buf) != 0;
+            boolean needReplaceInner = canGetRowFromInner && idx == cnt - 1 && io.getForward(pageAddr) != 0;
 
             // !!! Before modifying state we have to make sure that we will not go for retry.
 
             // We may need to replace inner key or want to merge this leaf with sibling after the remove -> keep lock.
             if (needReplaceInner ||
                 // We need to make sure that we have back or forward to be able to merge.
-                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(buf)))) {
+                ((r.fwdId != 0 || r.backId != 0) && mayMerge(cnt - 1, io.getMaxCount(pageAddr, pageSize())))) {
                 // If we have backId then we've already locked back page, nothing to do here.
                 if (r.fwdId != 0 && r.backId == 0) {
                     Result res = r.lockForward(0);
@@ -432,7 +431,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 if (needReplaceInner)
                     r.needReplaceInner = TRUE;
 
-                Tail<L> t = r.addTail(leaf, buf, io, 0, Tail.EXACT);
+                Tail<L> t = r.addTail(leaf, pageAddr, io, 0, Tail.EXACT);
 
                 t.idx = (short)idx;
 
@@ -441,18 +440,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 return FOUND;
             }
 
-            r.removeDataRowFromLeaf(leaf, io, buf, cnt, idx);
+            r.removeDataRowFromLeaf(leaf, io, pageAddr, cnt, idx);
 
             return FOUND;
         }
     };
 
     /** */
-    private final GetPageHandler<Remove> lockBackAndRemoveFromLeaf = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+    private final GetPageHandler<Remove> lockBackAndRmvFromLeaf = new GetPageHandler<Remove>() {
+        @Override protected Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
@@ -460,7 +459,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             // Keep locks on back and leaf pages for subsequent merges.
             if (res == FOUND && r.tail != null)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -468,17 +467,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Remove> lockBackAndTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page back, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page back, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             // Check that we have consistent view of the world.
-            if (io.getForward(buf) != r.pageId)
+            if (io.getForward(pageAddr) != r.pageId)
                 return RETRY;
 
             // Correct locking order: from back to forward.
             Result res = r.doLockTail(lvl);
 
             if (res == FOUND)
-                r.addTail(back, buf, io, lvl, Tail.BACK);
+                r.addTail(back, pageAddr, io, lvl, Tail.BACK);
 
             return res;
         }
@@ -486,9 +485,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Remove> lockTailForward = new GetPageHandler<Remove>() {
-        @Override protected Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override protected Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
-            r.addTail(page, buf, io, lvl, Tail.FORWARD);
+            r.addTail(page, pageAddr, io, lvl, Tail.FORWARD);
 
             return FOUND;
         }
@@ -496,12 +495,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final GetPageHandler<Remove> lockTail = new GetPageHandler<Remove>() {
-        @Override public Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, Remove r, int lvl)
+        @Override public Result run0(Page page, long pageAddr, BPlusIO<L> io, Remove r, int lvl)
             throws IgniteCheckedException {
             assert lvl > 0 : lvl; // We are not at the bottom.
 
             // Check that we have a correct view of the world.
-            if (io.getForward(buf) != r.fwdId)
+            if (io.getForward(pageAddr) != r.fwdId)
                 return RETRY;
 
             // We don't have a back page, need to lock our forward and become a back for it.
@@ -512,7 +511,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return res; // Retry.
             }
 
-            r.addTail(page, buf, io, lvl, Tail.EXACT);
+            r.addTail(page, pageAddr, io, lvl, Tail.EXACT);
 
             return FOUND;
         }
@@ -520,14 +519,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final PageHandler<Void, Bool> cutRoot = new PageHandler<Void, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Void ignore, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Void ignore, int lvl)
             throws IgniteCheckedException {
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getRootLevel(buf); // Can drop only root.
+            assert lvl == io.getRootLevel(pageAddr); // Can drop only root.
 
-            io.cutRoot(buf);
+            io.cutRoot(pageAddr, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageCutRootRecord(cacheId, meta.id()));
@@ -538,16 +537,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final PageHandler<Long, Bool> addRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootPageId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootPageId, int lvl)
             throws IgniteCheckedException {
             assert rootPageId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            assert lvl == io.getLevelsCount(buf);
+            assert lvl == io.getLevelsCount(pageAddr);
 
-            io.addRoot(buf, rootPageId);
+            io.addRoot(pageAddr, rootPageId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageAddRootRecord(cacheId, meta.id(), rootPageId));
@@ -558,14 +557,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /** */
     private final PageHandler<Long, Bool> initRoot = new PageHandler<Long, Bool>() {
-        @Override public Bool run(Page meta, PageIO iox, ByteBuffer buf, Long rootId, int lvl)
+        @Override public Bool run(Page meta, PageIO iox, long pageAddr, Long rootId, int lvl)
             throws IgniteCheckedException {
             assert rootId != null;
 
             // Safe cast because we should never recycle meta page until the tree is destroyed.
             BPlusMetaIO io = (BPlusMetaIO)iox;
 
-            io.initRoot(buf, rootId);
+            io.initRoot(pageAddr, rootId, pageSize());
 
             if (needWalDeltaRecord(meta))
                 wal.log(new MetaPageInitRootRecord(cacheId, meta.id(), rootId));
@@ -579,10 +578,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param cacheId Cache ID.
      * @param pageMem Page memory.
      * @param wal Write ahead log manager.
+     * @param globalRmvId Remove ID.
      * @param metaPageId Meta page ID.
      * @param reuseList Reuse list.
      * @param innerIos Inner IO versions.
      * @param leafIos Leaf IO versions.
+     * @throws IgniteCheckedException If failed.
      */
     public BPlusTree(
         String name,
@@ -641,28 +642,29 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long rootId = allocatePage(null);
 
         try (Page root = page(rootId)) {
-            initPage(root, this, latestLeafIO(), wal);
+            initPage(pageMem, root, this, latestLeafIO(), wal);
         }
 
         // Initialize meta page with new root page.
         try (Page meta = page(metaPageId)) {
-            Bool res = writePage(meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
+            Bool res = writePage(pageMem, meta, this, initRoot, BPlusMetaIO.VERSIONS.latest(), wal, rootId, 0, FALSE);
 
             assert res == TRUE: res;
         }
     }
 
     /**
+     * @param meta Meta page.
      * @return Root level.
      */
     private int getRootLevel(Page meta) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            return BPlusMetaIO.VERSIONS.forPage(buf).getRootLevel(buf);
+            return BPlusMetaIO.VERSIONS.forPage(pageAddr).getRootLevel(pageAddr);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
@@ -672,27 +674,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @return Page ID.
      */
     private long getFirstPageId(Page meta, int lvl) {
-        ByteBuffer buf = readLock(meta); // Meta can't be removed.
+        long pageAddr = readLock(meta); // Meta can't be removed.
 
         try {
-            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+            BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
             if (lvl < 0)
-                lvl = io.getRootLevel(buf);
+                lvl = io.getRootLevel(pageAddr);
 
-            if (lvl >= io.getLevelsCount(buf))
+            if (lvl >= io.getLevelsCount(pageAddr))
                 return 0;
 
-            return io.getFirstPageId(buf, lvl);
+            return io.getFirstPageId(pageAddr, lvl);
         }
         finally {
-            readUnlock(meta, buf);
+            readUnlock(meta, pageAddr);
         }
     }
 
     /**
      * @param upper Upper bound.
      * @return Cursor.
+     * @throws IgniteCheckedException If failed.
      */
     private GridCursor<T> findLowerUnbounded(L upper) throws IgniteCheckedException {
         ForwardCursor cursor = new ForwardCursor(null, upper);
@@ -704,13 +707,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         try (Page first = page(firstPageId)) {
-            ByteBuffer buf = readLock(first); // We always merge pages backwards, the first page is never removed.
+            long pageAddr = readLock(first); // We always merge pages backwards, the first page is never removed.
 
             try {
-                cursor.init(buf, io(buf), 0);
+                cursor.init(pageAddr, io(pageAddr), 0);
             }
             finally {
-                readUnlock(first, buf);
+                readUnlock(first, pageAddr);
             }
         }
 
@@ -758,6 +761,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * @param row Lookup row for exact match.
      * @return Found row.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     @Override public final T findOne(L row) throws IgniteCheckedException {
@@ -783,6 +787,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param g Get.
+     * @throws IgniteCheckedException If failed.
      */
     private void doFind(Get g) throws IgniteCheckedException {
         try {
@@ -921,22 +926,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private void validateDownKeys(long pageId, L minRow) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
 
                 if (io.isLeaf()) {
                     for (int i = 0; i < cnt; i++) {
-                        if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                        if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                             fail("Wrong sort order: " + U.hexLong(pageId) + " , at " + i + " , minRow: " + minRow);
 
-                        minRow = io.getLookupRow(this, buf, i);
+                        minRow = io.getLookupRow(this, pageAddr, i);
                     }
 
                     return;
@@ -944,16 +949,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
                 // To find our inner key we have to go left and then always go to the right.
                 for (int i = 0; i < cnt; i++) {
-                    L row = io.getLookupRow(this, buf, i);
+                    L row = io.getLookupRow(this, pageAddr, i);
 
-                    if (minRow != null && compare(io, buf, i, minRow) <= 0)
+                    if (minRow != null && compare(io, pageAddr, i, minRow) <= 0)
                         fail("Min row violated: " + row + " , minRow: " + minRow);
 
-                    long leftId = inner(io).getLeft(buf, i);
+                    long leftId = inner(io).getLeft(pageAddr, i);
 
                     L leafRow = getGreatestRowInSubTree(leftId);
 
-                    int cmp = compare(io, buf, i, leafRow);
+                    int cmp = compare(io, pageAddr, i, leafRow);
 
                     if (cmp < 0 || (cmp != 0 && canGetRowFromInner))
                         fail("Wrong inner row: " + U.hexLong(pageId) + " , at: " + i + " , leaf:  " + leafRow +
@@ -965,12 +970,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 }
 
                 // Need to handle the rightmost child subtree separately or handle empty routing page.
-                long rightId = inner(io).getLeft(buf, cnt); // The same as getRight(cnt - 1)
+                long rightId = inner(io).getLeft(pageAddr, cnt); // The same as getRight(cnt - 1)
 
                 validateDownKeys(rightId, minRow);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -982,26 +987,26 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private L getGreatestRowInSubTree(long pageId) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (io.isLeaf()) {
                     if (cnt <= 0) // This code is called only if the tree is not empty, so we can't see empty leaf.
                         fail("Invalid leaf count: " + cnt + " " + U.hexLong(pageId));
 
-                    return io.getLookupRow(this, buf, cnt - 1);
+                    return io.getLookupRow(this, pageAddr, cnt - 1);
                 }
 
-                long rightId = inner(io).getLeft(buf, cnt);// The same as getRight(cnt - 1), but good for routing pages.
+                long rightId = inner(io).getLeft(pageAddr, cnt);// The same as getRight(cnt - 1), but good for routing pages.
 
                 return getGreatestRowInSubTree(rightId);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
@@ -1037,18 +1042,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long leftmostChildId;
 
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
                 if (io.isLeaf())
                     fail("Leaf.");
 
-                leftmostChildId = inner(io).getLeft(buf, 0);
+                leftmostChildId = inner(io).getLeft(pageAddr, 0);
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
 
@@ -1067,25 +1072,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      */
     private void validateDownPages(Page meta, long pageId, long fwdId, final int lvl) throws IgniteCheckedException {
         try (Page page = page(pageId)) {
-            ByteBuffer buf = readLock(page); // No correctness guaranties.
+            long pageAddr = readLock(page); // No correctness guaranties.
 
             try {
-                long realPageId = BPlusIO.getPageId(buf);
+                long realPageId = BPlusIO.getPageId(pageAddr);
 
                 if (realPageId != pageId)
                     fail(new SB("ABA on page ID: ref ").appendHex(pageId).a(", buf ").appendHex(realPageId));
 
-                BPlusIO<L> io = io(buf);
+                BPlusIO<L> io = io(pageAddr);
 
                 if (io.isLeaf() != (lvl == 0)) // Leaf pages only at the level 0.
                     fail("Leaf level mismatch: " + lvl);
 
-                long actualFwdId = io.getForward(buf);
+                long actualFwdId = io.getForward(pageAddr);
 
                 if (actualFwdId != fwdId)
                     fail(new SB("Triangle: expected fwd ").appendHex(fwdId).a(", actual fwd ").appendHex(actualFwdId));
 
-                int cnt = io.getCount(buf);
+                int cnt = io.getCount(pageAddr);
 
                 if (cnt < 0)
                     fail("Negative count: " + cnt);
@@ -1097,66 +1102,66 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 else {
                     // Recursively go down if we are on inner level.
                     for (int i = 0; i < cnt; i++)
-                        validateDownPages(meta, inner(io).getLeft(buf, i), inner(io).getRight(buf, i), lvl - 1);
+                        validateDownPages(meta, inner(io).getLeft(pageAddr, i), inner(io).getRight(pageAddr, i), lvl - 1);
 
                     if (fwdId != 0) {
                         // For the rightmost child ask neighbor.
                         try (Page fwd = page(fwdId)) {
-                            ByteBuffer fwdBuf = readLock(fwd); // No correctness guaranties.
+                            long fwdPageAddr = readLock(fwd); // No correctness guaranties.
 
                             try {
-                                if (io(fwdBuf) != io)
+                                if (io(fwdPageAddr) != io)
                                     fail("IO on the same level must be the same");
 
-                                fwdId = inner(io).getLeft(fwdBuf, 0);
+                                fwdId = inner(io).getLeft(fwdPageAddr, 0);
                             }
                             finally {
-                                readUnlock(fwd, fwdBuf);
+                                readUnlock(fwd, fwdPageAddr);
                             }
                         }
                     }
 
-                    pageId = inner(io).getLeft(buf, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
+                    pageId = inner(io).getLeft(pageAddr, cnt); // The same as io.getRight(cnt - 1) but works for routing pages.
 
                     validateDownPages(meta, pageId, fwdId, lvl - 1);
                 }
             }
             finally {
-                readUnlock(page, buf);
+                readUnlock(page, pageAddr);
             }
         }
     }
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param keys Keys.
      * @return String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPage(BPlusIO<L> io, ByteBuffer buf, boolean keys) throws IgniteCheckedException {
+    private String printPage(BPlusIO<L> io, long pageAddr, boolean keys) throws IgniteCheckedException {
         StringBuilder b = new StringBuilder();
 
-        b.append(formatPageId(PageIO.getPageId(buf)));
+        b.append(formatPageId(PageIO.getPageId(pageAddr)));
 
         b.append(" [ ");
         b.append(io.isLeaf() ? "L " : "I ");
 
-        int cnt = io.getCount(buf);
-        long fwdId = io.getForward(buf);
+        int cnt = io.getCount(pageAddr);
+        long fwdId = io.getForward(pageAddr);
 
         b.append("cnt=").append(cnt).append(' ');
         b.append("fwd=").append(formatPageId(fwdId)).append(' ');
 
         if (!io.isLeaf()) {
-            b.append("lm=").append(formatPageId(inner(io).getLeft(buf, 0))).append(' ');
+            b.append("lm=").append(formatPageId(inner(io).getLeft(pageAddr, 0))).append(' ');
 
             if (cnt > 0)
-                b.append("rm=").append(formatPageId(inner(io).getRight(buf, cnt - 1))).append(' ');
+                b.append("rm=").append(formatPageId(inner(io).getRight(pageAddr, cnt - 1))).append(' ');
         }
 
         if (keys)
-            b.append("keys=").append(printPageKeys(io, buf)).append(' ');
+            b.append("keys=").append(printPageKeys(io, pageAddr)).append(' ');
 
         b.append(']');
 
@@ -1165,12 +1170,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Keys as String.
      * @throws IgniteCheckedException If failed.
      */
-    private String printPageKeys(BPlusIO<L> io, ByteBuffer buf) throws IgniteCheckedException {
-        int cnt = io.getCount(buf);
+    private String printPageKeys(BPlusIO<L> io, long pageAddr) throws IgniteCheckedException {
+        int cnt = io.getCount(pageAddr);
 
         StringBuilder b = new StringBuilder();
 
@@ -1180,7 +1185,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             if (i != 0)
                 b.append(',');
 
-            b.append(io.isLeaf() || canGetRowFromInner ? getRow(io, buf, i) : io.getLookupRow(this, buf, i));
+            b.append(io.isLeaf() || canGetRowFromInner ? getRow(io, pageAddr, i) : io.getLookupRow(this, pageAddr, i));
         }
 
         b.append(']');
@@ -1487,21 +1492,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         while (pageId != 0) {
             try (Page page = page(pageId)) {
-                ByteBuffer buf = readLock(page); // No correctness guaranties.
+                long pageAddr = readLock(page); // No correctness guaranties.
 
                 try {
                     if (io == null) {
-                        io = io(buf);
+                        io = io(pageAddr);
 
                         assert io.isLeaf();
                     }
 
-                    cnt += io.getCount(buf);
+                    cnt += io.getCount(pageAddr);
 
-                    pageId = io.getForward(buf);
+                    pageId = io.getForward(pageAddr);
                 }
                 finally {
-                    readUnlock(page, buf);
+                    readUnlock(page, pageAddr);
                 }
             }
         }
@@ -1531,9 +1536,9 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             for (;;) { // Go down with retries.
                 p.init();
 
-                Result result = putDown(p, p.rootId, 0L, p.rootLvl);
+                Result res = putDown(p, p.rootId, 0L, p.rootLvl);
 
-                switch (result) {
+                switch (res) {
                     case RETRY:
                     case RETRY_ROOT:
                         checkInterrupted();
@@ -1556,7 +1561,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         return p.oldRow;
 
                     default:
-                        throw new IllegalStateException("Result: " + result);
+                        throw new IllegalStateException("Result: " + res);
                 }
             }
         }
@@ -1595,28 +1600,28 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         long pagesCnt = 0;
 
         try (Page meta = page(metaPageId)) {
-            ByteBuffer metaBuf = writeLock(meta); // No checks, we must be out of use.
+            long metaPageAddr = writeLock(meta); // No checks, we must be out of use.
 
             try {
-                for (long pageId : getFirstPageIds(metaBuf)) {
+                for (long pageId : getFirstPageIds(metaPageAddr)) {
                     assert pageId != 0;
 
                     do {
                         try (Page page = page(pageId)) {
-                            ByteBuffer buf = writeLock(page); // No checks, we must be out of use.
+                            long pageAddr = writeLock(page); // No checks, we must be out of use.
 
                             try {
-                                BPlusIO<L> io = io(buf);
+                                BPlusIO<L> io = io(pageAddr);
 
-                                long fwdPageId = io.getForward(buf);
+                                long fwdPageId = io.getForward(pageAddr);
 
-                                bag.addFreePage(recyclePage(pageId, page, buf));
+                                bag.addFreePage(recyclePage(pageId, page, pageAddr));
                                 pagesCnt++;
 
                                 pageId = fwdPageId;
                             }
                             finally {
-                                writeUnlock(page, buf, true);
+                                writeUnlock(page, pageAddr, true);
                             }
                         }
 
@@ -1629,11 +1634,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     while (pageId != 0);
                 }
 
-                bag.addFreePage(recyclePage(metaPageId, meta, metaBuf));
+                bag.addFreePage(recyclePage(metaPageId, meta, metaPageAddr));
                 pagesCnt++;
             }
             finally {
-                writeUnlock(meta, metaBuf, true);
+                writeUnlock(meta, metaPageAddr, true);
             }
         }
 
@@ -1647,39 +1652,38 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     /**
      * @return {@code True} if state was changed.
      */
-    protected final boolean markDestroyed() {
+    private final boolean markDestroyed() {
         return destroyed.compareAndSet(false, true);
     }
 
     /**
-     * @param metaBuf Meta page buffer.
+     * @param pageAddr Meta page address.
      * @return First page IDs.
      */
-    protected Iterable<Long> getFirstPageIds(ByteBuffer metaBuf) {
-        List<Long> result = new ArrayList<>();
+    protected Iterable<Long> getFirstPageIds(long pageAddr) {
+        List<Long> res = new ArrayList<>();
 
-        BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(metaBuf);
+        BPlusMetaIO mio = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-        for (int lvl = mio.getRootLevel(metaBuf); lvl >= 0; lvl--) {
-            result.add(mio.getFirstPageId(metaBuf, lvl));
-        }
+        for (int lvl = mio.getRootLevel(pageAddr); lvl >= 0; lvl--)
+            res.add(mio.getFirstPageId(pageAddr, lvl));
 
-        return result;
+        return res;
     }
 
     /**
      * @param pageId Page ID.
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return Recycled page ID.
      * @throws IgniteCheckedException If failed.
      */
-    private long recyclePage(long pageId, Page page, ByteBuffer buf) throws IgniteCheckedException {
+    private long recyclePage(long pageId, Page page, long pageAddr) throws IgniteCheckedException {
         // Rotate page ID to avoid concurrency issues with reused pages.
         pageId = PageIdUtils.rotatePageId(pageId);
 
         // Update page ID inside of the buffer, Page.id() will always return the original page ID.
-        PageIO.setPageId(buf, pageId);
+        PageIO.setPageId(pageAddr, pageId);
 
         if (needWalDeltaRecord(page))
             wal.log(new RecycleRecord(cacheId, page.id(), pageId));
@@ -1701,10 +1705,10 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private boolean splitPage(
         BPlusIO io,
         Page page,
-        ByteBuffer buf,
+        long buf,
         long fwdId,
         Page fwd,
-        ByteBuffer fwdBuf,
+        long fwdBuf,
         int idx
     ) throws IgniteCheckedException {
         int cnt = io.getCount(buf);
@@ -1719,7 +1723,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         // Update forward page.
-        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt);
+        io.splitForwardPage(buf, fwdId, fwdBuf, mid, cnt, pageSize());
 
         // TODO GG-11640 log a correct forward page record.
         fwd.fullPageWalRecordPolicy(Boolean.TRUE);
@@ -1735,11 +1739,11 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param page Page.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      */
-    private void writeUnlockAndClose(Page page, ByteBuffer buf) {
+    private void writeUnlockAndClose(Page page, long pageAddr) {
         try {
-            writeUnlock(page, buf, true);
+            writeUnlock(page, pageAddr, true);
         }
         finally {
             page.close();
@@ -1798,7 +1802,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             p.fwdId = fwdId;
                             p.pageId = pageId;
 
-                            res = writePage(page, this, replace, p, lvl, RETRY);
+                            res = writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                             // Restore args.
                             p.pageId = oldPageId;
@@ -1828,7 +1832,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, replace, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, replace, p, lvl, RETRY);
 
                     case NOT_FOUND: // Do insert.
                         assert lvl == p.btmLvl : "must insert at the bottom level";
@@ -1838,7 +1842,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         p.pageId = pageId;
                         p.fwdId = fwdId;
 
-                        return writePage(page, this, insert, p, lvl, RETRY);
+                        return writePage(pageMem, page, this, insert, p, lvl, RETRY);
 
                     default:
                         return res;
@@ -1853,25 +1857,25 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
     /**
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param back Backward page.
      * @return Page ID.
      */
-    private long doAskNeighbor(BPlusIO<L> io, ByteBuffer buf, boolean back) {
+    private long doAskNeighbor(BPlusIO<L> io, long pageAddr, boolean back) {
         long res;
 
         if (back) {
             // Count can be 0 here if it is a routing page, in this case we have a single child.
-            int cnt = io.getCount(buf);
+            int cnt = io.getCount(pageAddr);
 
             // We need to do get the rightmost child: io.getRight(cnt - 1),
             // here io.getLeft(cnt) is the same, but handles negative index if count is 0.
-            res = inner(io).getLeft(buf, cnt);
+            res = inner(io).getLeft(pageAddr, cnt);
         }
         else // Leftmost child.
-            res = inner(io).getLeft(buf, 0);
+            res = inner(io).getLeft(pageAddr, 0);
 
-        assert res != 0 : "inner page with no route down: " + U.hexLong(PageIO.getPageId(buf));
+        assert res != 0 : "inner page with no route down: " + U.hexLong(PageIO.getPageId(pageAddr));
 
         return res;
     }
@@ -1935,19 +1939,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             int rootLvl;
             long rootId;
 
-            ByteBuffer buf = readLock(meta); // Meta can't be removed.
+            long pageAddr = readLock(meta); // Meta can't be removed.
 
-            assert buf != null : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
+            assert pageAddr != 0 : "Failed to read lock meta page [page=" + meta + ", metaPageId=" +
                 U.hexLong(metaPageId) + ']';
 
             try {
-                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(buf);
+                BPlusMetaIO io = BPlusMetaIO.VERSIONS.forPage(pageAddr);
 
-                rootLvl = io.getRootLevel(buf);
-                rootId = io.getFirstPageId(buf, rootLvl);
+                rootLvl = io.getRootLevel(pageAddr);
+                rootId = io.getFirstPageId(pageAddr, rootLvl);
             }
             finally {
-                readUnlock(meta, buf);
+                readUnlock(meta, pageAddr);
             }
 
             restartFromRoot(rootId, rootLvl, globalRmvId.get());
@@ -1966,13 +1970,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index of found entry.
          * @param lvl Level.
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -1980,13 +1984,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Insertion point.
          * @param lvl Level.
          * @return {@code true} If we need to stop.
          * @throws IgniteCheckedException If failed.
          */
-        boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             assert lvl >= 0;
 
             return lvl == 0; // Stop if we are at the bottom.
@@ -2024,12 +2028,12 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             // Check if we are on an inner page and can't get row from it.
             if (lvl != 0 && !canGetRowFromInner)
                 return false;
 
-            row = getRow(io, buf, idx);
+            row = getRow(io, pageAddr, idx);
 
             return true;
         }
@@ -2057,16 +2061,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             throw new IllegalStateException(); // Must never be called because we always have a shift.
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) throws IgniteCheckedException {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) throws IgniteCheckedException {
             if (lvl != 0)
                 return false;
 
-            cursor.init(buf, io, idx);
+            cursor.init(pageAddr, io, idx);
 
             return true;
         }
@@ -2090,7 +2094,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Page tail;
 
         /** */
-        private ByteBuffer tailBuf;
+        private long tailPageAddr;
 
         /**
          * Bottom level for insertion (insert can't go deeper). Will be incremented on split on each level.
@@ -2114,7 +2118,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean found(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean found(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             if (lvl == 0) // Leaf: need to stop.
                 return true;
 
@@ -2128,7 +2132,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             assert btmLvl >= 0 : btmLvl;
             assert lvl >= btmLvl : lvl;
 
@@ -2137,16 +2141,16 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param tail Tail page.
-         * @param tailBuf Tail buffer.
+         * @param tailPageAddr Tail page address.
          */
-        private void tail(Page tail, ByteBuffer tailBuf) {
-            assert (tail == null) == (tailBuf == null);
+        private void tail(Page tail, long tailPageAddr) {
+            assert (tail == null) == (tailPageAddr == 0L);
 
             if (this.tail != null)
-                writeUnlockAndClose(this.tail, this.tailBuf);
+                writeUnlockAndClose(this.tail, this.tailPageAddr);
 
             this.tail = tail;
-            this.tailBuf = tailBuf;
+            this.tailPageAddr = tailPageAddr;
         }
 
         /** {@inheritDoc} */
@@ -2161,7 +2165,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             row = null;
             rightId = 0;
 
-            tail(null, null);
+            tail(null, 0L);
         }
 
         /**
@@ -2174,21 +2178,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insert(Page page, BPlusIO<L> io, ByteBuffer buf, int idx, int lvl)
+        private L insert(Page page, BPlusIO<L> io, long pageAddr, int idx, int lvl)
             throws IgniteCheckedException {
-            int maxCnt = io.getMaxCount(buf);
-            int cnt = io.getCount(buf);
+            int maxCnt = io.getMaxCount(pageAddr, pageSize());
+            int cnt = io.getCount(pageAddr);
 
             if (cnt == maxCnt) // Need to split page.
-                return insertWithSplit(page, io, buf, idx, lvl);
+                return insertWithSplit(page, io, pageAddr, idx, lvl);
 
-            insertSimple(page, io, buf, idx);
+            insertSimple(page, io, pageAddr, idx);
 
             return null;
         }
@@ -2196,13 +2200,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @throws IgniteCheckedException If failed.
          */
-        private void insertSimple(Page page, BPlusIO<L> io, ByteBuffer buf, int idx)
+        private void insertSimple(Page page, BPlusIO<L> io, long pageAddr, int idx)
             throws IgniteCheckedException {
-            io.insert(buf, idx, row, null, rightId);
+            io.insert(pageAddr, idx, row, null, rightId);
 
             if (needWalDeltaRecord(page))
                 wal.log(new InsertRecord<>(cacheId, page.id(), io, idx, row, null, rightId));
@@ -2211,55 +2215,55 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param idx Index.
          * @param lvl Level.
          * @return Move up row.
          * @throws IgniteCheckedException If failed.
          */
-        private L insertWithSplit(Page page, BPlusIO<L> io, final ByteBuffer buf, int idx, int lvl)
+        private L insertWithSplit(Page page, BPlusIO<L> io, final long pageAddr, int idx, int lvl)
             throws IgniteCheckedException {
             long fwdId = allocatePage(bag);
 
             try (Page fwd = page(fwdId)) {
                 // Need to check this before the actual split, because after the split we will have new forward page here.
-                boolean hadFwd = io.getForward(buf) != 0;
+                boolean hadFwd = io.getForward(pageAddr) != 0;
 
-                ByteBuffer fwdBuf = writeLock(fwd); // Initial write, no need to check for concurrent modification.
+                long fwdPageAddr = writeLock(fwd); // Initial write, no need to check for concurrent modification.
 
-                assert fwdBuf != null;
+                assert fwdPageAddr != 0L;
 
                 try {
                     // Never write full forward page, because it is known to be new.
                     fwd.fullPageWalRecordPolicy(Boolean.FALSE);
 
-                    boolean midShift = splitPage(io, page, buf, fwdId, fwd, fwdBuf, idx);
+                    boolean midShift = splitPage(io, page, pageAddr, fwdId, fwd, fwdPageAddr, idx);
 
                     // Do insert.
-                    int cnt = io.getCount(buf);
+                    int cnt = io.getCount(pageAddr);
 
                     if (idx < cnt || (idx == cnt && !midShift)) { // Insert into back page.
-                        insertSimple(page, io, buf, idx);
+                        insertSimple(page, io, pageAddr, idx);
 
                         // Fix leftmost child of forward page, because newly inserted row will go up.
                         if (idx == cnt && !io.isLeaf()) {
-                            inner(io).setLeft(fwdBuf, 0, rightId);
+                            inner(io).setLeft(fwdPageAddr, 0, rightId);
 
                             if (needWalDeltaRecord(fwd)) // Rare case, we can afford separate WAL record to avoid complexity.
                                 wal.log(new FixLeftmostChildRecord(cacheId, fwd.id(), rightId));
                         }
                     }
                     else // Insert into newly allocated forward page.
-                        insertSimple(fwd, io, fwdBuf, idx - cnt);
+                        insertSimple(fwd, io, fwdPageAddr, idx - cnt);
 
                     // Do move up.
-                    cnt = io.getCount(buf);
+                    cnt = io.getCount(pageAddr);
 
                     // Last item from backward row goes up.
-                    L moveUpRow = io.getLookupRow(BPlusTree.this, buf, cnt - 1);
+                    L moveUpRow = io.getLookupRow(BPlusTree.this, pageAddr, cnt - 1);
 
                     if (!io.isLeaf()) { // Leaf pages must contain all the links, inner pages remove moveUpLink.
-                        io.setCount(buf, cnt - 1);
+                        io.setCount(pageAddr, cnt - 1);
 
                         if (needWalDeltaRecord(page)) // Rare case, we can afford separate WAL record to avoid complexity.
                             wal.log(new FixCountRecord(cacheId, page.id(), cnt - 1));
@@ -2272,28 +2276,34 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                             if (io.isLeaf())
                                 io = latestInnerIO();
 
-                            ByteBuffer newRootBuf = writeLock(newRoot); // Initial write.
+                            long newRootPageAddr = writeLock(newRoot); // Initial write.
 
-                            assert newRootBuf != null;
+                            assert newRootPageAddr != 0L;
 
                             try {
                                 // Never write full new root page, because it is known to be new.
                                 newRoot.fullPageWalRecordPolicy(Boolean.FALSE);
 
-                                long pageId = PageIO.getPageId(buf);
+                                long pageId = PageIO.getPageId(pageAddr);
 
-                                inner(io).initNewRoot(newRootBuf, newRootId, pageId, moveUpRow, null, fwdId);
+                                inner(io).initNewRoot(newRootPageAddr,
+                                    newRootId,
+                                    pageId,
+                                    moveUpRow,
+                                    null,
+                                    fwdId,
+                                    pageSize());
 
                                 if (needWalDeltaRecord(newRoot))
                                     wal.log(new NewRootInitRecord<>(cacheId, newRoot.id(), newRootId,
                                         inner(io), pageId, moveUpRow, null, fwdId));
                             }
                             finally {
-                                writeUnlock(newRoot, newRootBuf, true);
+                                writeUnlock(newRoot, newRootPageAddr, true);
                             }
                         }
 
-                        Bool res = writePage(meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
+                        Bool res = writePage(pageMem, meta, BPlusTree.this, addRoot, newRootId, lvl + 1, FALSE);
 
                         assert res == TRUE: res;
 
@@ -2304,7 +2314,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                     return moveUpRow;
                 }
                 finally {
-                    writeUnlock(fwd, fwdBuf, true);
+                    writeUnlock(fwd, fwdPageAddr, true);
                 }
             }
         }
@@ -2402,7 +2412,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /** {@inheritDoc} */
-        @Override boolean notFound(BPlusIO<L> io, ByteBuffer buf, int idx, int lvl) {
+        @Override boolean notFound(BPlusIO<L> io, long pageAddr, int idx, int lvl) {
             if (lvl == 0) {
                 assert tail == null;
 
@@ -2622,7 +2632,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                         // Exit: we are done.
                     }
                     else if (tail.sibling != null &&
-                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf)) {
+                        tail.getCount() + tail.sibling.getCount() < tail.io.getMaxCount(tail.buf, pageSize())) {
                         // Release everything lower than tail, we've already merged this path.
                         doReleaseTail(tail.down);
                         tail.down = null;
@@ -2678,7 +2688,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndRemoveFromLeaf, this, 0, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndRmvFromLeaf, this, 0, RETRY);
             }
             finally {
                 if (canRelease(back, 0))
@@ -2693,7 +2703,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Result doRemoveFromLeaf() throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, removeFromLeaf, this, 0, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, rmvFromLeaf, this, 0, RETRY);
         }
 
         /**
@@ -2704,7 +2714,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private Result doLockTail(int lvl) throws IgniteCheckedException {
             assert page != null;
 
-            return writePage(page, BPlusTree.this, lockTail, this, lvl, RETRY);
+            return writePage(pageMem, page, BPlusTree.this, lockTail, this, lvl, RETRY);
         }
 
         /**
@@ -2732,7 +2742,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page back = page(backId);
 
             try {
-                return writePage(back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
+                return writePage(pageMem, back, BPlusTree.this, lockBackAndTail, this, lvl, RETRY);
             }
             finally {
                 if (canRelease(back, lvl))
@@ -2752,7 +2762,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
             Page fwd = page(fwdId);
 
             try {
-                return writePage(fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
+                return writePage(pageMem, fwd, BPlusTree.this, lockTailForward, this, lvl, RETRY);
             }
             finally {
                 // If we were not able to lock forward page as tail, release the page.
@@ -2764,21 +2774,21 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void removeDataRowFromLeaf(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
             throws IgniteCheckedException {
             assert idx >= 0 && idx < cnt: idx;
             assert io.isLeaf(): "inner";
             assert !isRemoved(): "already removed";
 
             // Detach the row.
-            removed = getRow(io, buf, idx);
+            removed = getRow(io, pageAddr, idx);
 
-            doRemove(page, io, buf, cnt, idx);
+            doRemove(page, io, pageAddr, cnt, idx);
 
             assert isRemoved();
         }
@@ -2786,17 +2796,17 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         /**
          * @param page Page.
          * @param io IO.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param cnt Count.
          * @param idx Index to remove.
          * @throws IgniteCheckedException If failed.
          */
-        private void doRemove(Page page, BPlusIO<L> io, ByteBuffer buf, int cnt, int idx)
+        private void doRemove(Page page, BPlusIO<L> io, long pageAddr, int cnt, int idx)
             throws IgniteCheckedException {
             assert cnt > 0 : cnt;
             assert idx >= 0 && idx < cnt : idx + " " + cnt;
 
-            io.remove(buf, idx, cnt);
+            io.remove(pageAddr, idx, cnt);
 
             if (needWalDeltaRecord(page))
                 wal.log(new RemoveRecord(cacheId, page.id(), idx, cnt));
@@ -2937,7 +2947,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             boolean emptyBranch = needMergeEmptyBranch == TRUE || needMergeEmptyBranch == READY;
 
-            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch))
+            if (!left.io.merge(prnt.io, prnt.buf, prntIdx, left.buf, right.buf, emptyBranch, pageSize()))
                 return false;
 
             // Invalidate indexes after successful merge.
@@ -2959,23 +2969,23 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param release Release write lock and release page.
          * @throws IgniteCheckedException If failed.
          */
-        private void freePage(Page page, ByteBuffer buf, boolean release)
+        private void freePage(Page page, long pageAddr, boolean release)
             throws IgniteCheckedException {
             long pageId = page.id();
 
             long effectivePageId = PageIdUtils.effectivePageId(pageId);
 
-            pageId = recyclePage(pageId, page, buf);
+            pageId = recyclePage(pageId, page, pageAddr);
 
             if (effectivePageId != PageIdUtils.effectivePageId(pageId))
                 throw new IllegalStateException("Effective page ID must stay the same.");
 
             if (release)
-                writeUnlockAndClose(page, buf);
+                writeUnlockAndClose(page, pageAddr);
 
             bag().addFreePage(pageId);
         }
@@ -2985,7 +2995,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         private void cutRoot(int lvl) throws IgniteCheckedException {
-            Bool res = writePage(meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
+            Bool res = writePage(pageMem, meta, BPlusTree.this, cutRoot, null, lvl, FALSE);
 
             assert res == TRUE: res;
         }
@@ -3156,14 +3166,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param lvl Level.
          * @param type Type.
          * @return Added tail.
          */
-        private Tail<L> addTail(Page page, ByteBuffer buf, BPlusIO<L> io, int lvl, byte type) {
-            final Tail<L> t = new Tail<>(page, buf, io, type, lvl);
+        private Tail<L> addTail(Page page, long pageAddr, BPlusIO<L> io, int lvl, byte type) {
+            final Tail<L> t = new Tail<>(page, pageAddr, io, type, lvl);
 
             if (tail == null)
                 tail = t;
@@ -3270,7 +3280,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         private final Page page;
 
         /** */
-        private final ByteBuffer buf;
+        private final long buf;
 
         /** */
         private final BPlusIO<L> io;
@@ -3297,7 +3307,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @param type Type.
          * @param lvl Level.
          */
-        private Tail(Page page, ByteBuffer buf, BPlusIO<L> io, byte type, int lvl) {
+        private Tail(Page page, long buf, BPlusIO<L> io, byte type, int lvl) {
             assert type == BACK || type == EXACT || type == FORWARD : type;
             assert lvl >= 0 && lvl <= Byte.MAX_VALUE : lvl;
             assert page != null;
@@ -3350,7 +3360,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * @param shift Shift if equal.
      * @return Insertion point as in {@link Arrays#binarySearch(Object[], Object, Comparator)}.
      */
-    private int findInsertionPoint(BPlusIO<L> io, ByteBuffer buf, int low, int cnt, L row, int shift)
+    private int findInsertionPoint(BPlusIO<L> io, long buf, int low, int cnt, L row, int shift)
         throws IgniteCheckedException {
         assert row != null;
 
@@ -3377,14 +3387,14 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @return IO.
      */
-    private BPlusIO<L> io(ByteBuffer buf) {
-        assert buf != null;
+    private BPlusIO<L> io(long pageAddr) {
+        assert pageAddr != 0;
 
-        int type = PageIO.getType(buf);
-        int ver = PageIO.getVersion(buf);
+        int type = PageIO.getType(pageAddr);
+        int ver = PageIO.getVersion(pageAddr);
 
         if (innerIos.getType() == type)
             return innerIos.forVersion(ver);
@@ -3392,7 +3402,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         if (leafIos.getType() == type)
             return leafIos.forVersion(ver);
 
-        throw new IllegalStateException("Unknown page type: " + type + " pageId: " + U.hexLong(PageIO.getPageId(buf)));
+        throw new IllegalStateException("Unknown page type: " + type + " pageId: " + U.hexLong(PageIO.getPageId(pageAddr)));
     }
 
     /**
@@ -3420,24 +3430,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     }
 
     /**
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index of row in the given buffer.
      * @param row Lookup row.
      * @return Comparison result as in {@link Comparator#compare(Object, Object)}.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract int compare(BPlusIO<L> io, ByteBuffer buf, int idx, L row) throws IgniteCheckedException;
+    protected abstract int compare(BPlusIO<L> io, long pageAddr, int idx, L row) throws IgniteCheckedException;
 
     /**
      * Get the full detached row. Can be called on inner page only if {@link #canGetRowFromInner} is {@code true}.
      *
      * @param io IO.
-     * @param buf Buffer.
+     * @param pageAddr Page address.
      * @param idx Index.
      * @return Full detached data row.
      * @throws IgniteCheckedException If failed.
      */
-    protected abstract T getRow(BPlusIO<L> io, ByteBuffer buf, int idx) throws IgniteCheckedException;
+    protected abstract T getRow(BPlusIO<L> io, long pageAddr, int idx) throws IgniteCheckedException;
 
     /**
      * Forward cursor.
@@ -3472,24 +3482,24 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param startIdx Start index.
          * @throws IgniteCheckedException If failed.
          */
-        private void init(ByteBuffer buf, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
+        private void init(long pageAddr, BPlusIO<L> io, int startIdx) throws IgniteCheckedException {
             nextPageId = 0;
             row = -1;
 
-            int cnt = io.getCount(buf);
+            int cnt = io.getCount(pageAddr);
 
             // If we see an empty page here, it means that it is an empty tree.
             if (cnt == 0) {
-                assert io.getForward(buf) == 0L;
+                assert io.getForward(pageAddr) == 0L;
 
                 rows = null;
             }
-            else if (!fillFromBuffer(buf, io, startIdx, cnt)) {
+            else if (!fillFromBuffer(pageAddr, io, startIdx, cnt)) {
                 if (rows != EMPTY) {
                     assert rows.length > 0; // Otherwise it makes no sense to create an array.
 
@@ -3500,18 +3510,18 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param cnt Count.
          * @return Adjusted to lower bound start index.
          * @throws IgniteCheckedException If failed.
          */
-        private int findLowerBound(ByteBuffer buf, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
+        private int findLowerBound(long pageAddr, BPlusIO<L> io, int cnt) throws IgniteCheckedException {
             // Compare with the first row on the page.
-            int cmp = compare(io, buf, 0, lowerBound);
+            int cmp = compare(io, pageAddr, 0, lowerBound);
 
             if (cmp < 0 || (cmp == 0 && lowerShift == 1)) {
-                int idx = findInsertionPoint(io, buf, 0, cnt, lowerBound, lowerShift);
+                int idx = findInsertionPoint(io, pageAddr, 0, cnt, lowerBound, lowerShift);
 
                 assert idx < 0;
 
@@ -3522,19 +3532,19 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param low Start index.
          * @param cnt Number of rows in the buffer.
          * @return Corrected number of rows with respect to upper bound.
          * @throws IgniteCheckedException If failed.
          */
-        private int findUpperBound(ByteBuffer buf, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
+        private int findUpperBound(long pageAddr, BPlusIO<L> io, int low, int cnt) throws IgniteCheckedException {
             // Compare with the last row on the page.
-            int cmp = compare(io, buf, cnt - 1, upperBound);
+            int cmp = compare(io, pageAddr, cnt - 1, upperBound);
 
             if (cmp > 0) {
-                int idx = findInsertionPoint(io, buf, low, cnt, upperBound, 1);
+                int idx = findInsertionPoint(io, pageAddr, low, cnt, upperBound, 1);
 
                 assert idx < 0;
 
@@ -3547,7 +3557,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
         }
 
         /**
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param startIdx Start index.
          * @param cnt Number of rows in the buffer.
@@ -3555,7 +3565,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
          * @throws IgniteCheckedException If failed.
          */
         @SuppressWarnings("unchecked")
-        private boolean fillFromBuffer(ByteBuffer buf, BPlusIO<L> io, int startIdx, int cnt)
+        private boolean fillFromBuffer(long pageAddr, BPlusIO<L> io, int startIdx, int cnt)
             throws IgniteCheckedException {
             assert io.isLeaf() : io;
             assert cnt != 0 : cnt; // We can not see empty pages (empty tree handled in init).
@@ -3564,13 +3574,13 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
 
             checkDestroyed();
 
-            nextPageId = io.getForward(buf);
+            nextPageId = io.getForward(pageAddr);
 
             if (lowerBound != null && startIdx == 0)
-                startIdx = findLowerBound(buf, io, cnt);
+                startIdx = findLowerBound(pageAddr, io, cnt);
 
             if (upperBound != null && cnt != startIdx)
-                cnt = findUpperBound(buf, io, startIdx, cnt);
+                cnt = findUpperBound(pageAddr, io, startIdx, cnt);
 
             cnt -= startIdx;
 
@@ -3581,7 +3591,7 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 rows = (T[])new Object[cnt];
 
             for (int i = 0; i < cnt; i++) {
-                T r = getRow(io, buf, startIdx + i);
+                T r = getRow(io, pageAddr, startIdx + i);
 
                 rows = GridArrays.set(rows, i, r);
             }
@@ -3662,22 +3672,22 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
                 }
 
                 try (Page next = page(nextPageId)) {
-                    ByteBuffer buf = readLock(next); // Doing explicit null check.
+                    long pageAddr = readLock(next); // Doing explicit null check.
 
                     // If concurrent merge occurred we have to reinitialize cursor from the last returned row.
-                    if (buf == null)
+                    if (pageAddr == 0L)
                         break;
 
                     try {
-                        BPlusIO<L> io = io(buf);
+                        BPlusIO<L> io = io(pageAddr);
 
-                        if (fillFromBuffer(buf, io, 0, io.getCount(buf)))
+                        if (fillFromBuffer(pageAddr, io, 0, io.getCount(pageAddr)))
                             return true;
 
                         // Continue fetching forward.
                     }
                     finally {
-                        readUnlock(next, buf);
+                        readUnlock(next, pageAddr);
                     }
                 }
             }
@@ -3712,31 +3722,31 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
     private abstract class GetPageHandler<G extends Get> extends PageHandler<G, Result> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
-        @Override public final Result run(Page page, PageIO iox, ByteBuffer buf, G g, int lvl)
+        @Override public final Result run(Page page, PageIO iox, long pageAddr, G g, int lvl)
             throws IgniteCheckedException {
-            assert PageIO.getPageId(buf) == page.id();
+            assert PageIO.getPageId(pageAddr) == page.id();
 
             // If we've passed the check for correct page ID, we can safely cast.
             BPlusIO<L> io = (BPlusIO<L>)iox;
 
             // In case of intersection with inner replace in remove operation
             // we need to restart our operation from the tree root.
-            if (lvl == 0 && g.rmvId < io.getRemoveId(buf))
+            if (lvl == 0 && g.rmvId < io.getRemoveId(pageAddr))
                 return RETRY_ROOT;
 
-            return run0(page, buf, io, g, lvl);
+            return run0(page, pageAddr, io, g, lvl);
         }
 
         /**
          * @param page Page.
-         * @param buf Buffer.
+         * @param pageAddr Page address.
          * @param io IO.
          * @param g Operation.
          * @param lvl Level.
          * @return Result code.
          * @throws IgniteCheckedException If failed.
          */
-        protected abstract Result run0(Page page, ByteBuffer buf, BPlusIO<L> io, G g, int lvl)
+        protected abstract Result run0(Page page, long pageAddr, BPlusIO<L> io, G g, int lvl)
             throws IgniteCheckedException;
 
         /** {@inheritDoc} */
@@ -3774,13 +3784,39 @@ public abstract class BPlusTree<L, T extends L> extends DataStructure implements
      * Operation result.
      */
     enum Result {
-        GO_DOWN, GO_DOWN_X, FOUND, NOT_FOUND, RETRY, RETRY_ROOT
+        /** */
+        GO_DOWN,
+
+        /** */
+        GO_DOWN_X,
+
+        /** */
+        FOUND,
+
+        /** */
+        NOT_FOUND,
+
+        /** */
+        RETRY,
+
+        /** */
+        RETRY_ROOT
     }
 
     /**
      * Four state boolean.
      */
     enum Bool {
-        FALSE, TRUE, READY, DONE
+        /** */
+        FALSE,
+
+        /** */
+        TRUE,
+
+        /** */
+        READY,
+
+        /** */
+        DONE
     }
 }